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 vv...@apache.org on 2016/06/13 09:27:21 UTC

[01/51] [abbrv] hadoop git commit: YARN-4837. User facing aspects of 'AM blacklisting' feature need fixing. (vinodkv via wangda) [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/YARN-3926 97db7eb3d -> 7a0d5db2d (forced update)


http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.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/AppAttemptInfo.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/AppAttemptInfo.java
index 60b728e..e8c8bca 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/AppAttemptInfo.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/AppAttemptInfo.java
@@ -42,7 +42,7 @@ public class AppAttemptInfo {
   protected String nodeId;
   protected String logsLink;
   protected String blacklistedNodes;
-  protected String rmBlacklistedNodesForAMLaunches;
+  private String nodesBlacklistedBySystem;
   protected String appAttemptId;
 
   public AppAttemptInfo() {
@@ -69,9 +69,9 @@ public class AppAttemptInfo {
             + masterContainer.getNodeHttpAddress(),
             ConverterUtils.toString(masterContainer.getId()), user);
 
-        rmBlacklistedNodesForAMLaunches = StringUtils.join(
-            attempt.getAMBlacklist().getBlacklistUpdates().getAdditions(),
-            ", ");
+        nodesBlacklistedBySystem =
+            StringUtils.join(attempt.getAMBlacklistManager()
+              .getBlacklistUpdates().getBlacklistAdditions(), ", ");
         if (rm.getResourceScheduler() instanceof AbstractYarnScheduler) {
           AbstractYarnScheduler ayScheduler =
               (AbstractYarnScheduler) rm.getResourceScheduler();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ApplicationSubmissionContextInfo.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/ApplicationSubmissionContextInfo.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/ApplicationSubmissionContextInfo.java
index 4cbe7a8..3d95ca1 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/ApplicationSubmissionContextInfo.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/ApplicationSubmissionContextInfo.java
@@ -87,9 +87,6 @@ public class ApplicationSubmissionContextInfo {
   @XmlElement(name = "reservation-id")
   String reservationId;
 
-  @XmlElement(name = "am-black-listing-requests")
-  AMBlackListingRequestInfo amBlackListingRequestInfo;
-
   public ApplicationSubmissionContextInfo() {
     applicationId = "";
     applicationName = "";
@@ -106,7 +103,6 @@ public class ApplicationSubmissionContextInfo {
     logAggregationContextInfo = null;
     attemptFailuresValidityInterval = -1;
     reservationId = "";
-    amBlackListingRequestInfo = null;
   }
 
   public String getApplicationId() {
@@ -173,10 +169,6 @@ public class ApplicationSubmissionContextInfo {
     return attemptFailuresValidityInterval;
   }
 
-  public AMBlackListingRequestInfo getAMBlackListingRequestInfo() {
-    return amBlackListingRequestInfo;
-  }
-
   public String getReservationId() {
     return reservationId;
   }
@@ -252,9 +244,4 @@ public class ApplicationSubmissionContextInfo {
   public void setReservationId(String reservationId) {
     this.reservationId = reservationId;
   }
-
-  public void setAMBlackListingRequestInfo(
-      AMBlackListingRequestInfo amBlackListingRequestInfo) {
-    this.amBlackListingRequestInfo = amBlackListingRequestInfo;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.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/TestNodeBlacklistingOnAMFailures.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.java
new file mode 100644
index 0000000..ef6d43b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.java
@@ -0,0 +1,251 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager;
+
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.DrainDispatcher;
+import org.apache.hadoop.yarn.event.EventDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager.TestAMRestart;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Validate system behavior when the am-scheduling logic 'blacklists' a node for
+ * an application because of AM failures.
+ */
+public class TestNodeBlacklistingOnAMFailures {
+
+  @Test(timeout = 100000)
+  public void testNodeBlacklistingOnAMFailure() throws Exception {
+
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    conf.setBoolean(YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_ENABLED,
+        true);
+
+    DrainDispatcher dispatcher = new DrainDispatcher();
+    MockRM rm = startRM(conf, dispatcher);
+    CapacityScheduler scheduler = (CapacityScheduler) rm.getResourceScheduler();
+
+    MockNM nm1 =
+        new MockNM("127.0.0.1:1234", 8000, rm.getResourceTrackerService());
+    nm1.registerNode();
+
+    MockNM nm2 =
+        new MockNM("127.0.0.2:2345", 8000, rm.getResourceTrackerService());
+    nm2.registerNode();
+
+    RMApp app = rm.submitApp(200);
+
+    MockAM am1 = MockRM.launchAndRegisterAM(app, rm, nm1);
+    ContainerId amContainerId =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
+    RMContainer rmContainer = scheduler.getRMContainer(amContainerId);
+    NodeId nodeWhereAMRan = rmContainer.getAllocatedNode();
+
+    MockNM currentNode, otherNode;
+    if (nodeWhereAMRan.equals(nm1.getNodeId())) {
+      currentNode = nm1;
+      otherNode = nm2;
+    } else {
+      currentNode = nm2;
+      otherNode = nm1;
+    }
+
+    // Set the exist status to INVALID so that we can verify that the system
+    // automatically blacklisting the node
+    makeAMContainerExit(rm, amContainerId, currentNode,
+        ContainerExitStatus.INVALID);
+
+    // restart the am
+    RMAppAttempt attempt = MockRM.waitForAttemptScheduled(app, rm);
+    System.out.println("New AppAttempt launched " + attempt.getAppAttemptId());
+
+    // Try the current node a few times
+    for (int i = 0; i <= 2; i++) {
+      currentNode.nodeHeartbeat(true);
+      dispatcher.await();
+
+      Assert.assertEquals(
+          "AppAttemptState should still be SCHEDULED if currentNode is "
+              + "blacklisted correctly", RMAppAttemptState.SCHEDULED,
+          attempt.getAppAttemptState());
+    }
+
+    // Now try the other node
+    otherNode.nodeHeartbeat(true);
+    dispatcher.await();
+
+    // Now the AM container should be allocated
+    MockRM.waitForState(attempt, RMAppAttemptState.ALLOCATED, 20000);
+
+    MockAM am2 = rm.sendAMLaunched(attempt.getAppAttemptId());
+    rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED);
+    amContainerId =
+        ContainerId.newContainerId(am2.getApplicationAttemptId(), 1);
+    rmContainer = scheduler.getRMContainer(amContainerId);
+    nodeWhereAMRan = rmContainer.getAllocatedNode();
+
+    // The other node should now receive the assignment
+    Assert.assertEquals(
+        "After blacklisting, AM should have run on the other node",
+        otherNode.getNodeId(), nodeWhereAMRan);
+
+    am2.registerAppAttempt();
+    rm.waitForState(app.getApplicationId(), RMAppState.RUNNING);
+
+    List<Container> allocatedContainers =
+        TestAMRestart.allocateContainers(currentNode, am2, 1);
+    Assert.assertEquals(
+        "Even though AM is blacklisted from the node, application can "
+            + "still allocate non-AM containers there",
+        currentNode.getNodeId(), allocatedContainers.get(0).getNodeId());
+  }
+
+  @Test(timeout = 100000)
+  public void testNoBlacklistingForNonSystemErrors() throws Exception {
+
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_ENABLED,
+        true);
+    // disable the float so it is possible to blacklist the entire cluster
+    conf.setFloat(
+        YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD,
+        1.5f);
+    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 100);
+
+    DrainDispatcher dispatcher = new DrainDispatcher();
+    MockRM rm = startRM(conf, dispatcher);
+
+    MockNM node =
+        new MockNM("127.0.0.1:1234", 8000, rm.getResourceTrackerService());
+    node.registerNode();
+
+    RMApp app = rm.submitApp(200);
+    ApplicationId appId = app.getApplicationId();
+
+    int numAppAttempts = 1;
+
+    // Now the AM container should be allocated
+    RMAppAttempt attempt = MockRM.waitForAttemptScheduled(app, rm);
+    node.nodeHeartbeat(true);
+    dispatcher.await();
+    MockRM.waitForState(attempt, RMAppAttemptState.ALLOCATED, 20000);
+    rm.sendAMLaunched(attempt.getAppAttemptId());
+    rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED);
+    ApplicationAttemptId appAttemptId =
+        ApplicationAttemptId.newInstance(appId, numAppAttempts);
+    ContainerId amContainerId = ContainerId.newContainerId(appAttemptId, 1);
+
+    for (int containerExitStatus : new int[] {
+        ContainerExitStatus.PREEMPTED,
+        ContainerExitStatus.KILLED_BY_RESOURCEMANAGER,
+        // ContainerExitStatus.KILLED_BY_APPMASTER,
+        ContainerExitStatus.KILLED_AFTER_APP_COMPLETION,
+        ContainerExitStatus.ABORTED, ContainerExitStatus.DISKS_FAILED,
+        ContainerExitStatus.KILLED_EXCEEDED_VMEM,
+        ContainerExitStatus.KILLED_EXCEEDED_PMEM }) {
+
+      // Set the exist status to be containerExitStatus so that we can verify
+      // that the system automatically blacklisting the node
+      makeAMContainerExit(rm, amContainerId, node, containerExitStatus);
+
+      // restart the am
+      attempt = MockRM.waitForAttemptScheduled(app, rm);
+      System.out
+          .println("New AppAttempt launched " + attempt.getAppAttemptId());
+
+      node.nodeHeartbeat(true);
+      dispatcher.await();
+
+      MockRM.waitForState(attempt, RMAppAttemptState.ALLOCATED, 20000);
+      rm.sendAMLaunched(attempt.getAppAttemptId());
+      rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED);
+
+      numAppAttempts++;
+      appAttemptId = ApplicationAttemptId.newInstance(appId, numAppAttempts);
+      amContainerId = ContainerId.newContainerId(appAttemptId, 1);
+      rm.waitForState(node, amContainerId, RMContainerState.ACQUIRED);
+    }
+  }
+
+  private void makeAMContainerExit(MockRM rm, ContainerId amContainer,
+      MockNM node, int exitStatus) throws Exception, InterruptedException {
+    ContainerStatus containerStatus =
+        BuilderUtils.newContainerStatus(amContainer, ContainerState.COMPLETE,
+            "", exitStatus, Resources.createResource(200));
+    node.containerStatus(containerStatus);
+    ApplicationAttemptId amAttemptID = amContainer.getApplicationAttemptId();
+    rm.waitForState(amAttemptID, RMAppAttemptState.FAILED);
+    rm.waitForState(amAttemptID.getApplicationId(), RMAppState.ACCEPTED);
+  }
+
+  private MockRM startRM(YarnConfiguration conf,
+      final DrainDispatcher dispatcher) {
+
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(conf);
+
+    MockRM rm1 = new MockRM(conf, memStore) {
+      @Override
+      protected EventHandler<SchedulerEvent> createSchedulerEventDispatcher() {
+        return new EventDispatcher<SchedulerEvent>(this.scheduler,
+            this.scheduler.getClass().getName()) {
+          @Override
+          public void handle(SchedulerEvent event) {
+            super.handle(event);
+          }
+        };
+      }
+
+      @Override
+      protected Dispatcher createDispatcher() {
+        return dispatcher;
+      }
+    };
+
+    rm1.start();
+    return rm1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/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 66780a0..76a17c3 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
@@ -35,13 +35,8 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
-import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.event.Dispatcher;
-import org.apache.hadoop.yarn.event.DrainDispatcher;
-import org.apache.hadoop.yarn.event.EventDispatcher;
-import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
@@ -54,18 +49,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
-import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.Records;
-import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -238,7 +229,7 @@ public class TestAMRestart {
     rm1.stop();
   }
 
-  private List<Container> allocateContainers(MockNM nm1, MockAM am1,
+  public static List<Container> allocateContainers(MockNM nm1, MockAM am1,
       int NUM_CONTAINERS) throws Exception {
     // allocate NUM_CONTAINERS containers
     am1.allocate("127.0.0.1", 1024, NUM_CONTAINERS,
@@ -277,7 +268,9 @@ public class TestAMRestart {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 3);
     // To prevent test from blacklisting nm1 for AM, we sit threshold to half
     // of 2 nodes which is 1
-    conf.setFloat(YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD, 0.5f);
+    conf.setFloat(
+        YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD,
+        0.5f);
 
     MockRM rm1 = new MockRM(conf);
     rm1.start();
@@ -379,168 +372,6 @@ public class TestAMRestart {
     rm1.stop();
   }
 
-  @Test(timeout = 100000)
-  public void testAMBlacklistPreventsRestartOnSameNode() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
-        ResourceScheduler.class);
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    testAMBlacklistPreventRestartOnSameNode(false, conf);
-  }
-
-  @Test(timeout = 100000)
-  public void testAMBlacklistPreventsRestartOnSameNodeForMinicluster()
-      throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    conf.setBoolean(YarnConfiguration.RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME,
-        true);
-    testAMBlacklistPreventRestartOnSameNode(false, conf);
-  }
-
-  @Test(timeout = 100000)
-  public void testAMBlacklistPreemption() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    // disable the float so it is possible to blacklist the entire cluster
-    conf.setFloat(YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD, 1.5f);
-    // since the exit status is PREEMPTED, it should not lead to the node being
-    // blacklisted
-    testAMBlacklistPreventRestartOnSameNode(true, conf);
-  }
-
-  /**
-   * Tests AM blacklisting. In the multi-node mode (i.e. singleNode = false),
-   * it tests the blacklisting behavior so that the AM container gets allocated
-   * on the node that is not blacklisted. In the single-node mode, it tests the
-   * PREEMPTED status to see if the AM container can continue to be scheduled.
-   */
-  private void testAMBlacklistPreventRestartOnSameNode(boolean singleNode,
-      YarnConfiguration conf) throws Exception {
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    final DrainDispatcher dispatcher = new DrainDispatcher();
-    MockRM rm1 = new MockRM(conf, memStore) {
-      @Override
-      protected EventHandler<SchedulerEvent> createSchedulerEventDispatcher() {
-        return new EventDispatcher<SchedulerEvent>(this.scheduler,
-            this.scheduler.getClass().getName()) {
-          @Override
-          public void handle(SchedulerEvent event) {
-            super.handle(event);
-          }
-        };
-      }
-
-      @Override
-      protected Dispatcher createDispatcher() {
-        return dispatcher;
-      }
-    };
-
-    rm1.start();
-
-    MockNM nm1 =
-        new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());
-    nm1.registerNode();
-
-    MockNM nm2 = null;
-    if (!singleNode) {
-      nm2 =
-          new MockNM("127.0.0.2:2345", 8000, rm1.getResourceTrackerService());
-      nm2.registerNode();
-    }
-
-    RMApp app1 = rm1.submitApp(200);
-
-    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-    CapacityScheduler scheduler =
-        (CapacityScheduler) rm1.getResourceScheduler();
-    ContainerId amContainer =
-        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
-    // Preempt the first attempt;
-    RMContainer rmContainer = scheduler.getRMContainer(amContainer);
-    NodeId nodeWhereAMRan = rmContainer.getAllocatedNode();
-
-    MockNM currentNode, otherNode;
-    if (singleNode) {
-      Assert.assertEquals(nm1.getNodeId(), nodeWhereAMRan);
-      currentNode = nm1;
-      otherNode = null; // not applicable
-    } else {
-      if (nodeWhereAMRan == nm1.getNodeId()) {
-        currentNode = nm1;
-        otherNode = nm2;
-      } else {
-        currentNode = nm2;
-        otherNode = nm1;
-      }
-    }
-
-    // set the exist status to test
-    // any status other than SUCCESS and PREEMPTED should cause the node to be
-    // blacklisted
-    int exitStatus = singleNode ?
-            ContainerExitStatus.PREEMPTED :
-            ContainerExitStatus.INVALID;
-    ContainerStatus containerStatus =
-        BuilderUtils.newContainerStatus(amContainer, ContainerState.COMPLETE,
-            "", exitStatus, Resources.createResource(200));
-    currentNode.containerStatus(containerStatus);
-    rm1.waitForState(am1.getApplicationAttemptId(), RMAppAttemptState.FAILED);
-    rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
-
-    // restart the am
-    RMAppAttempt attempt = MockRM.waitForAttemptScheduled(app1, rm1);
-    System.out.println("Launch AM " + attempt.getAppAttemptId());
-
-
-
-    currentNode.nodeHeartbeat(true);
-    dispatcher.await();
-
-    if (!singleNode) {
-      Assert.assertEquals(
-          "AppAttemptState should still be SCHEDULED if currentNode is " +
-          "blacklisted correctly",
-          RMAppAttemptState.SCHEDULED,
-          attempt.getAppAttemptState());
-
-      otherNode.nodeHeartbeat(true);
-      dispatcher.await();
-    }
-
-    MockAM am2 = rm1.sendAMLaunched(attempt.getAppAttemptId());
-    rm1.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED);
-    amContainer =
-        ContainerId.newContainerId(am2.getApplicationAttemptId(), 1);
-    rmContainer = scheduler.getRMContainer(amContainer);
-    nodeWhereAMRan = rmContainer.getAllocatedNode();
-    if (singleNode) {
-      // with preemption, the node should not be blacklisted and should get the
-      // assignment (with a single node)
-      Assert.assertEquals(
-          "AM should still have been able to run on the same node",
-          currentNode.getNodeId(), nodeWhereAMRan);
-    } else {
-      // with a failed status, the other node should receive the assignment
-      Assert.assertEquals(
-          "After blacklisting AM should have run on the other node",
-          otherNode.getNodeId(), nodeWhereAMRan);
-
-      am2.registerAppAttempt();
-      rm1.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
-
-      List<Container> allocatedContainers =
-          allocateContainers(currentNode, am2, 1);
-      Assert.assertEquals(
-          "Even though AM is blacklisted from the node, application can " +
-          "still allocate containers there",
-          currentNode.getNodeId(), allocatedContainers.get(0).getNodeId());
-    }
-  }
-
-
   // AM container preempted, nm disk failure
   // should not be counted towards AM max retry count.
   @Test(timeout = 100000)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.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/blacklist/TestBlacklistManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java
index 96b373f..3102c6f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/TestBlacklistManager.java
@@ -19,12 +19,13 @@
 package org.apache.hadoop.yarn.server.resourcemanager.blacklist;
 
 
-import org.junit.Assert;
-import org.junit.Test;
-
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
+import org.junit.Assert;
+import org.junit.Test;
+
 public class TestBlacklistManager {
 
   @Test
@@ -37,12 +38,12 @@ public class TestBlacklistManager {
     String anyNode2 = "bar";
     manager.addNode(anyNode);
     manager.addNode(anyNode2);
-    BlacklistUpdates blacklist = manager
+    ResourceBlacklistRequest blacklist = manager
         .getBlacklistUpdates();
 
-    List<String> blacklistAdditions = blacklist.getAdditions();
+    List<String> blacklistAdditions = blacklist.getBlacklistAdditions();
     Collections.sort(blacklistAdditions);
-    List<String> blacklistRemovals = blacklist.getRemovals();
+    List<String> blacklistRemovals = blacklist.getBlacklistRemovals();
     String[] expectedBlacklistAdditions = new String[]{anyNode2, anyNode};
     Assert.assertArrayEquals(
         "Blacklist additions was not as expected",
@@ -61,12 +62,12 @@ public class TestBlacklistManager {
     String anyNode = "foo";
     String anyNode2 = "bar";
     manager.addNode(anyNode);
-    BlacklistUpdates blacklist = manager
+    ResourceBlacklistRequest blacklist = manager
         .getBlacklistUpdates();
 
-    List<String> blacklistAdditions = blacklist.getAdditions();
+    List<String> blacklistAdditions = blacklist.getBlacklistAdditions();
     Collections.sort(blacklistAdditions);
-    List<String> blacklistRemovals = blacklist.getRemovals();
+    List<String> blacklistRemovals = blacklist.getBlacklistRemovals();
     String[] expectedBlacklistAdditions = new String[]{anyNode};
     Assert.assertArrayEquals(
         "Blacklist additions was not as expected",
@@ -81,9 +82,9 @@ public class TestBlacklistManager {
 
     blacklist = manager
         .getBlacklistUpdates();
-    blacklistAdditions = blacklist.getAdditions();
+    blacklistAdditions = blacklist.getBlacklistAdditions();
     Collections.sort(blacklistAdditions);
-    blacklistRemovals = blacklist.getRemovals();
+    blacklistRemovals = blacklist.getBlacklistRemovals();
     Collections.sort(blacklistRemovals);
     String[] expectedBlacklistRemovals = new String[] {anyNode2, anyNode};
     Assert.assertTrue(
@@ -101,11 +102,11 @@ public class TestBlacklistManager {
     BlacklistManager disabled = new DisabledBlacklistManager();
     String anyNode = "foo";
     disabled.addNode(anyNode);
-    BlacklistUpdates blacklist = disabled
+    ResourceBlacklistRequest blacklist = disabled
         .getBlacklistUpdates();
 
-    List<String> blacklistAdditions = blacklist.getAdditions();
-    List<String> blacklistRemovals = blacklist.getRemovals();
+    List<String> blacklistAdditions = blacklist.getBlacklistAdditions();
+    List<String> blacklistRemovals = blacklist.getBlacklistRemovals();
     Assert.assertTrue(
         "Blacklist additions should be empty but was " +
             blacklistAdditions,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.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/TestRMAppTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
index 293c0b6..bb5c25c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.yarn.MockApps;
-import org.apache.hadoop.yarn.api.records.AMBlackListingRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
@@ -1029,63 +1028,6 @@ public class TestRMAppTransitions {
             + "/"));
   }
 
-  @Test
-  public void testAMBlackListConfigFromApp() {
-    // Scenario 1: Application enables AM blacklisting
-    float disableThreshold = 0.9f;
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, false);
-    ApplicationSubmissionContext submissionContext =
-        new ApplicationSubmissionContextPBImpl();
-    submissionContext.setAMBlackListRequest(AMBlackListingRequest.newInstance(
-        true, disableThreshold));
-    RMAppImpl application = (RMAppImpl) createNewTestApp(submissionContext);
-
-    Assert.assertTrue(application.isAmBlacklistingEnabled());
-    Assert.assertEquals(disableThreshold,
-        application.getAmBlacklistingDisableThreshold(), 1e-8);
-
-    // Scenario 2: Application disables AM blacklisting
-    float globalThreshold = 0.9f;
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    conf.setFloat(YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD,
-        globalThreshold);
-    ApplicationSubmissionContext submissionContext2 =
-        new ApplicationSubmissionContextPBImpl();
-    submissionContext2.setAMBlackListRequest(AMBlackListingRequest.newInstance(
-        false, disableThreshold));
-    RMAppImpl application2 = (RMAppImpl) createNewTestApp(submissionContext2);
-
-    // Am blacklisting will be disabled eventhough its enabled in RM.
-    Assert.assertFalse(application2.isAmBlacklistingEnabled());
-
-    // Scenario 3: Application updates invalid AM threshold
-    float invalidDisableThreshold = -0.5f;
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    conf.setFloat(YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD,
-        globalThreshold);
-    ApplicationSubmissionContext submissionContext3 =
-        new ApplicationSubmissionContextPBImpl();
-    submissionContext3.setAMBlackListRequest(AMBlackListingRequest.newInstance(
-        true, invalidDisableThreshold));
-    RMAppImpl application3 = (RMAppImpl) createNewTestApp(submissionContext3);
-
-    Assert.assertTrue(application3.isAmBlacklistingEnabled());
-    Assert.assertEquals(globalThreshold,
-        application3.getAmBlacklistingDisableThreshold(), 1e-8);
-
-    // Scenario 4: Empty AMBlackListingRequest in Submission Context
-    conf.setBoolean(YarnConfiguration.AM_BLACKLISTING_ENABLED, true);
-    conf.setFloat(YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD,
-        globalThreshold);
-    ApplicationSubmissionContext submissionContext4 =
-        new ApplicationSubmissionContextPBImpl();
-    RMAppImpl application4 = (RMAppImpl) createNewTestApp(submissionContext4);
-
-    Assert.assertTrue(application4.isAmBlacklistingEnabled());
-    Assert.assertEquals(globalThreshold,
-        application4.getAmBlacklistingDisableThreshold(), 1e-8);
-  }
-
   private void verifyApplicationFinished(RMAppState state) {
     ArgumentCaptor<RMAppState> finalState =
         ArgumentCaptor.forClass(RMAppState.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.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/TestAppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
index 4141a53..a1c6294 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
@@ -41,32 +41,32 @@ public class TestAppSchedulingInfo {
     AppSchedulingInfo  appSchedulingInfo = new AppSchedulingInfo(
         appAttemptId, "test", queue, null, 0, new ResourceUsage());
 
-    appSchedulingInfo.updateBlacklist(new ArrayList<String>(),
+    appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
         new ArrayList<String>());
     Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged());
 
     ArrayList<String> blacklistAdditions = new ArrayList<String>();
     blacklistAdditions.add("node1");
     blacklistAdditions.add("node2");
-    appSchedulingInfo.updateBlacklist(blacklistAdditions,
+    appSchedulingInfo.updatePlacesBlacklistedByApp(blacklistAdditions,
         new ArrayList<String>());
     Assert.assertTrue(appSchedulingInfo.getAndResetBlacklistChanged());
 
     blacklistAdditions.clear();
     blacklistAdditions.add("node1");
-    appSchedulingInfo.updateBlacklist(blacklistAdditions,
+    appSchedulingInfo.updatePlacesBlacklistedByApp(blacklistAdditions,
         new ArrayList<String>());
     Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged());
 
     ArrayList<String> blacklistRemovals = new ArrayList<String>();
     blacklistRemovals.add("node1");
-    appSchedulingInfo.updateBlacklist(new ArrayList<String>(),
+    appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
         blacklistRemovals);
-    appSchedulingInfo.updateBlacklist(new ArrayList<String>(),
+    appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
         blacklistRemovals);
     Assert.assertTrue(appSchedulingInfo.getAndResetBlacklistChanged());
 
-    appSchedulingInfo.updateBlacklist(new ArrayList<String>(),
+    appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
         blacklistRemovals);
     Assert.assertFalse(appSchedulingInfo.getAndResetBlacklistChanged());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
index 72d2f85..7c34292 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
@@ -684,14 +684,16 @@ public class TestCapacityScheduler {
     cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(),
         Collections.singletonList(host), null, null, null);
-    Assert.assertTrue(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
+    Assert.assertTrue(cs.getApplicationAttempt(appAttemptId)
+        .isPlaceBlacklisted(host));
     cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(), null,
         Collections.singletonList(host), null, null);
-    Assert.assertFalse(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
+    Assert.assertFalse(cs.getApplicationAttempt(appAttemptId)
+        .isPlaceBlacklisted(host));
     rm.stop();
   }
-  
+
   @Test
   public void testAllocateReorder() throws Exception {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
index 1901fa9..af1dc62 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppAttempt.java
@@ -314,24 +314,24 @@ public class TestFSAppAttempt extends FairSchedulerTestBase {
     FSAppAttempt spyApp = spy(app);
     doReturn(false)
         .when(spyApp).isWaitingForAMContainer();
-    assertTrue(spyApp.isBlacklisted(n1.getNodeName()));
-    assertFalse(spyApp.isBlacklisted(n2.getNodeName()));
+    assertTrue(spyApp.isPlaceBlacklisted(n1.getNodeName()));
+    assertFalse(spyApp.isPlaceBlacklisted(n2.getNodeName()));
     assertEquals(n2.getUnallocatedResource(), spyApp.getHeadroom());
 
     blacklistAdditions.clear();
     blacklistAdditions.add(n2.getNodeName());
     blacklistRemovals.add(n1.getNodeName());
     app.updateBlacklist(blacklistAdditions, blacklistRemovals);
-    assertFalse(spyApp.isBlacklisted(n1.getNodeName()));
-    assertTrue(spyApp.isBlacklisted(n2.getNodeName()));
+    assertFalse(spyApp.isPlaceBlacklisted(n1.getNodeName()));
+    assertTrue(spyApp.isPlaceBlacklisted(n2.getNodeName()));
     assertEquals(n1.getUnallocatedResource(), spyApp.getHeadroom());
 
     blacklistAdditions.clear();
     blacklistRemovals.clear();
     blacklistRemovals.add(n2.getNodeName());
     app.updateBlacklist(blacklistAdditions, blacklistRemovals);
-    assertFalse(spyApp.isBlacklisted(n1.getNodeName()));
-    assertFalse(spyApp.isBlacklisted(n2.getNodeName()));
+    assertFalse(spyApp.isPlaceBlacklisted(n1.getNodeName()));
+    assertFalse(spyApp.isPlaceBlacklisted(n2.getNodeName()));
     assertEquals(clusterResource, spyApp.getHeadroom());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index abf87d7..3e5a40f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -3995,11 +3995,12 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(),
         Collections.singletonList(host), null, null, null);
-    assertTrue(app.isBlacklisted(host));
+    assertTrue(app.isPlaceBlacklisted(host));
     scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(), null,
         Collections.singletonList(host), null, null);
-    assertFalse(scheduler.getSchedulerApp(appAttemptId).isBlacklisted(host));
+    assertFalse(scheduler.getSchedulerApp(appAttemptId)
+        .isPlaceBlacklisted(host));
 
     List<ResourceRequest> update = Arrays.asList(
         createResourceRequest(GB, node.getHostName(), 1, 0, true));
@@ -4008,7 +4009,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     scheduler.allocate(appAttemptId, update,
         Collections.<ContainerId>emptyList(),
         Collections.singletonList(host), null, null, null);
-    assertTrue(app.isBlacklisted(host));
+    assertTrue(app.isPlaceBlacklisted(host));
     scheduler.update();
     scheduler.handle(updateEvent);
     assertEquals("Incorrect number of containers allocated", 0, app
@@ -4018,7 +4019,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     scheduler.allocate(appAttemptId, update,
         Collections.<ContainerId>emptyList(), null,
         Collections.singletonList(host), null, null);
-    assertFalse(app.isBlacklisted(host));
+    assertFalse(app.isPlaceBlacklisted(host));
     createSchedulingRequest(GB, "root.default", "user", 1);
     scheduler.update();
     scheduler.handle(updateEvent);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.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/TestRMWebServicesAppsModification.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
index 38b32e9..c7ef8fa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
@@ -22,7 +22,14 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assume.assumeTrue;
 
-import java.io.*;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringReader;
+import java.io.StringWriter;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -43,15 +50,12 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
 
-import com.sun.jersey.api.client.config.DefaultClientConfig;
-
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
-import org.apache.hadoop.yarn.api.records.AMBlackListingRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
@@ -72,7 +76,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.*;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppPriority;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CredentialsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LocalResourceInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LogAggregationContextInfo;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
@@ -101,6 +111,7 @@ import com.sun.jersey.api.client.Client;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
 import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
 import com.sun.jersey.api.client.filter.LoggingFilter;
 import com.sun.jersey.api.json.JSONJAXBContext;
 import com.sun.jersey.api.json.JSONMarshaller;
@@ -827,17 +838,6 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
         System.currentTimeMillis(), 1).toString();
     appInfo.setReservationId(reservationId);
 
-    // Set AMBlackListingRequestInfo
-    boolean isAMBlackListingEnabled = true;
-    float disableFailureThreshold = 0.01f;
-    AMBlackListingRequestInfo amBlackListingRequestInfo
-        = new AMBlackListingRequestInfo();
-    amBlackListingRequestInfo.setAMBlackListingEnabled(
-        isAMBlackListingEnabled);
-    amBlackListingRequestInfo.setBlackListingDisableFailureThreshold(
-        disableFailureThreshold);
-    appInfo.setAMBlackListingRequestInfo(amBlackListingRequestInfo);
-
     ClientResponse response =
         this.constructWebResource(urlPath).accept(acceptMedia)
           .entity(appInfo, contentMedia).post(ClientResponse.class);
@@ -912,13 +912,6 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
     // Check ReservationId
     assertEquals(reservationId, app.getReservationId().toString());
 
-    // Check AMBlackListingRequestInfo
-    AMBlackListingRequest amBlackListingRequest = asc.getAMBlackListRequest();
-    assertEquals(isAMBlackListingEnabled,
-        amBlackListingRequest.isAMBlackListingEnabled());
-    assertTrue(disableFailureThreshold == amBlackListingRequest
-        .getBlackListingDisableFailureThreshold());
-
     response =
         this.constructWebResource("apps", appId).accept(acceptMedia)
           .get(ClientResponse.class);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[31/51] [abbrv] hadoop git commit: YARN-3426. Add jdiff support to YARN. (vinodkv via wangda)

Posted by vv...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/03fc6b1b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_2.7.2.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_2.7.2.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_2.7.2.xml
new file mode 100644
index 0000000..158528d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_2.7.2.xml
@@ -0,0 +1,2581 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Thu May 12 17:48:36 PDT 2016 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="hadoop-yarn-client 2.7.2"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.ExcludePrivateAnnotationsJDiffDoclet -docletpath /Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/hadoop-annotations.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/jdiff.jar -verbose -classpath /Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/classes:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-common-project/hadoop-common/target/hadoop-common-2.7.2.jar:/Users/vinodkv/.m2/repository/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/Users/vinodkv/.m2/repository/xmlenc/xmlenc/0.52/xmlenc-0.52.jar:/Users/vinodkv/.m2/repository/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1.jar:/Users/vinodkv/.m2/repository/commons-codec/commons-codec/1.4/comm
 ons-codec-1.4.jar:/Users/vinodkv/.m2/repository/commons-io/commons-io/2.4/commons-io-2.4.jar:/Users/vinodkv/.m2/repository/commons-net/commons-net/3.1/commons-net-3.1.jar:/Users/vinodkv/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/Users/vinodkv/.m2/repository/javax/servlet/servlet-api/2.5/servlet-api-2.5.jar:/Users/vinodkv/.m2/repository/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/Users/vinodkv/.m2/repository/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/Users/vinodkv/.m2/repository/javax/servlet/jsp/jsp-api/2.1/jsp-api-2.1.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/jersey-core/1.9/jersey-core-1.9.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/jersey-json/1.9/jersey-json-1.9.jar:/Users/vinodkv/.m2/repository/com/sun/xml/bind/jaxb-impl/2.2.3-1/jaxb-impl-2.2.3-1.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/jersey-server/1.9/jersey-server-1.9.jar:/Users/vinodkv/.m2/repository/asm/asm/3.2/asm-3.2.jar:/Users/vin
 odkv/.m2/repository/net/java/dev/jets3t/jets3t/0.9.0/jets3t-0.9.0.jar:/Users/vinodkv/.m2/repository/org/apache/httpcomponents/httpclient/4.2.5/httpclient-4.2.5.jar:/Users/vinodkv/.m2/repository/org/apache/httpcomponents/httpcore/4.2.5/httpcore-4.2.5.jar:/Users/vinodkv/.m2/repository/com/jamesmurty/utils/java-xmlbuilder/0.4/java-xmlbuilder-0.4.jar:/Users/vinodkv/.m2/repository/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/Users/vinodkv/.m2/repository/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/Users/vinodkv/.m2/repository/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/Users/vinodkv/.m2/repository/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar:/Users/vinodkv/.m2/repository/org/slf4j/slf4j-api/1.7.10/slf4j-api-1.7.10.jar:/Users/vinodkv/.m2/repository/org/slf4j/slf4j-log4j12/1.7.10/slf4j-log4j12-1.7.10.jar:/Users/vinodkv/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.1
 3/jackson-core-asl-1.9.13.jar:/Users/vinodkv/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/vinodkv/.m2/repository/org/apache/avro/avro/1.7.4/avro-1.7.4.jar:/Users/vinodkv/.m2/repository/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/Users/vinodkv/.m2/repository/org/xerial/snappy/snappy-java/1.0.4.1/snappy-java-1.0.4.1.jar:/Users/vinodkv/.m2/repository/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar:/Users/vinodkv/.m2/repository/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-common-project/hadoop-auth/target/hadoop-auth-2.7.2.jar:/Users/vinodkv/.m2/repository/org/apache/directory/server/apacheds-kerberos-codec/2.0.0-M15/apacheds-kerberos-codec-2.0.0-M15.jar:/Users/vinodkv/.m2/repository/org/apache/directory/server/apacheds-i18n/2.0.0-M15/apacheds-i18n-2.0.0-M15.jar:/Users/vinodkv/.m2/repository/org/apache/directory/api/api-asn1-api/1.0.
 0-M20/api-asn1-api-1.0.0-M20.jar:/Users/vinodkv/.m2/repository/org/apache/directory/api/api-util/1.0.0-M20/api-util-1.0.0-M20.jar:/Users/vinodkv/.m2/repository/org/apache/curator/curator-framework/2.7.1/curator-framework-2.7.1.jar:/Users/vinodkv/.m2/repository/com/jcraft/jsch/0.1.42/jsch-0.1.42.jar:/Users/vinodkv/.m2/repository/org/apache/curator/curator-client/2.7.1/curator-client-2.7.1.jar:/Users/vinodkv/.m2/repository/org/apache/curator/curator-recipes/2.7.1/curator-recipes-2.7.1.jar:/Users/vinodkv/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/vinodkv/.m2/repository/org/apache/htrace/htrace-core/3.1.0-incubating/htrace-core-3.1.0-incubating.jar:/Users/vinodkv/.m2/repository/org/apache/zookeeper/zookeeper/3.4.6/zookeeper-3.4.6.jar:/Users/vinodkv/.m2/repository/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/Users/vinodkv/.m2/repository/org/tukaani/xz/1.0/xz-1.0.jar:/Users/vinodkv/.m2/repository/com/google/guava/guava/11.0.2/guav
 a-11.0.2.jar:/Users/vinodkv/.m2/repository/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/Users/vinodkv/.m2/repository/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/Users/vinodkv/.m2/repository/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/Users/vinodkv/.m2/repository/log4j/log4j/1.2.17/log4j-1.2.17.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-common-project/hadoop-annotations/target/hadoop-annotations-2.7.2.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_45.jdk/Contents/Home/lib/tools.jar:/Users/vinodkv/.m2/repository/io/netty/netty/3.6.2.Final/netty-3.6.2.Final.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/hadoop-yarn-api-2.7.2.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/hadoop-yarn-common-2.7.2.jar:/Users/vinodkv/.m2/repository/javax/xml/bind/jaxb-api/2.2.2/jaxb-api-2.2.2.jar:/
 Users/vinodkv/.m2/repository/javax/xml/stream/stax-api/1.0-2/stax-api-1.0-2.jar:/Users/vinodkv/.m2/repository/javax/activation/activation/1.1/activation-1.1.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/jersey-client/1.9/jersey-client-1.9.jar:/Users/vinodkv/.m2/repository/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/Users/vinodkv/.m2/repository/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/Users/vinodkv/.m2/repository/com/google/inject/extensions/guice-servlet/3.0/guice-servlet-3.0.jar:/Users/vinodkv/.m2/repository/com/google/inject/guice/3.0/guice-3.0.jar:/Users/vinodkv/.m2/repository/javax/inject/javax.inject/1/javax.inject-1.jar:/Users/vinodkv/.m2/repository/aopalliance/aopalliance/1.0/aopalliance-1.0.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/contribs/jersey-guice/1.9/jersey-guice-1.9.jar:/Users/vinodkv/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar -sourcepath /Users/vinodkv/Workspace/eclipse-workspace/apache-g
 it/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java -apidir /Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/site/jdiff/xml -apiname hadoop-yarn-client 2.7.2 -->
+<package name="org.apache.hadoop.yarn.client">
+  <!-- start class org.apache.hadoop.yarn.client.SCMAdmin -->
+  <class name="SCMAdmin" extends="org.apache.hadoop.conf.Configured"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.util.Tool"/>
+    <constructor name="SCMAdmin"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="SCMAdmin" type="org.apache.hadoop.conf.Configuration"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createSCMAdminProtocol" return="org.apache.hadoop.yarn.server.api.SCMAdminProtocol"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="run" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="args" type="java.lang.String[]"/>
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="main"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="args" type="java.lang.String[]"/>
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.SCMAdmin -->
+</package>
+<package name="org.apache.hadoop.yarn.client.api">
+  <!-- start class org.apache.hadoop.yarn.client.api.AHSClient -->
+  <class name="AHSClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AHSClient" type="java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createAHSClient" return="org.apache.hadoop.yarn.client.api.AHSClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new instance of AHSClient.]]>
+      </doc>
+    </method>
+    <method name="getApplicationReport" return="org.apache.hadoop.yarn.api.records.ApplicationReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Get a report of the given Application.
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ <p>
+ If the user does not have <code>VIEW_APP</code> access then the following
+ fields in the report will be set to stubbed values:
+ <ul>
+   <li>host - set to "N/A"</li>
+   <li>RPC port - set to -1</li>
+   <li>client token - set to "N/A"</li>
+   <li>diagnostics - set to "N/A"</li>
+   <li>tracking URL - set to "N/A"</li>
+   <li>original tracking URL - set to "N/A"</li>
+   <li>resource usage report - all values are -1</li>
+ </ul>
+
+ @param appId
+          {@link ApplicationId} of the application that needs a report
+ @return application report
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of all Applications in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @return a list of reports for all applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttemptReport" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given ApplicationAttempt.
+ </p>
+
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+
+ @param applicationAttemptId
+          {@link ApplicationAttemptId} of the application attempt that needs
+          a report
+ @return application attempt report
+ @throws YarnException
+ @throws ApplicationAttemptNotFoundException if application attempt
+         not found
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttempts" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of all (ApplicationAttempts) of Application in the cluster.
+ </p>
+
+ @param applicationId
+ @return a list of reports for all application attempts for specified
+         application
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainerReport" return="org.apache.hadoop.yarn.api.records.ContainerReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given Container.
+ </p>
+
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+
+ @param containerId
+          {@link ContainerId} of the container that needs a report
+ @return container report
+ @throws YarnException
+ @throws ContainerNotFoundException if container not found
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainers" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of all (Containers) of ApplicationAttempt in the cluster.
+ </p>
+
+ @param applicationAttemptId
+ @return a list of reports of all containers for specified application
+         attempt
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.AHSClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.AMRMClient -->
+  <class name="AMRMClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AMRMClient" type="java.lang.String"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createAMRMClient" return="org.apache.hadoop.yarn.client.api.AMRMClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new instance of AMRMClient.
+ For usage:
+ <pre>
+ {@code
+ AMRMClient.<T>createAMRMClientContainerRequest()
+ }</pre>
+ @return the newly create AMRMClient instance.]]>
+      </doc>
+    </method>
+    <method name="registerApplicationMaster" return="org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appHostName" type="java.lang.String"/>
+      <param name="appHostPort" type="int"/>
+      <param name="appTrackingUrl" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Register the application master. This must be called before any
+ other interaction
+ @param appHostName Name of the host on which master is running
+ @param appHostPort Port master is listening on
+ @param appTrackingUrl URL at which the master info can be seen
+ @return <code>RegisterApplicationMasterResponse</code>
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="allocate" return="org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="progressIndicator" type="float"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Request additional containers and receive new container allocations.
+ Requests made via <code>addContainerRequest</code> are sent to the
+ <code>ResourceManager</code>. New containers assigned to the master are
+ retrieved. Status of completed containers and node health updates are also
+ retrieved. This also doubles up as a heartbeat to the ResourceManager and
+ must be made periodically. The call may not always return any new
+ allocations of containers. App should not make concurrent allocate
+ requests. May cause request loss.
+
+ <p>
+ Note : If the user has not removed container requests that have already
+ been satisfied, then the re-register may end up sending the entire
+ container requests to the RM (including matched requests). Which would mean
+ the RM could end up giving it a lot of new allocated containers.
+ </p>
+
+ @param progressIndicator Indicates progress made by the master
+ @return the response of the allocate request
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="unregisterApplicationMaster"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appStatus" type="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"/>
+      <param name="appMessage" type="java.lang.String"/>
+      <param name="appTrackingUrl" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Unregister the application master. This must be called in the end.
+ @param appStatus Success/Failure status of the master
+ @param appMessage Diagnostics message on failure
+ @param appTrackingUrl New URL to get master info
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="addContainerRequest"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="T"/>
+      <doc>
+      <![CDATA[Request containers for resources before calling <code>allocate</code>
+ @param req Resource request]]>
+      </doc>
+    </method>
+    <method name="removeContainerRequest"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="T"/>
+      <doc>
+      <![CDATA[Remove previous container request. The previous container request may have
+ already been sent to the ResourceManager. So even after the remove request
+ the app must be prepared to receive an allocation for the previous request
+ even after the remove request
+ @param req Resource request]]>
+      </doc>
+    </method>
+    <method name="releaseAssignedContainer"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <doc>
+      <![CDATA[Release containers assigned by the Resource Manager. If the app cannot use
+ the container or wants to give up the container then it can release them.
+ The app needs to make new requests for the released resource capability if
+ it still needs it. eg. it released non-local resources
+ @param containerId]]>
+      </doc>
+    </method>
+    <method name="getAvailableResources" return="org.apache.hadoop.yarn.api.records.Resource"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the currently available resources in the cluster.
+ A valid value is available after a call to allocate has been made
+ @return Currently available resources]]>
+      </doc>
+    </method>
+    <method name="getClusterNodeCount" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the current number of nodes in the cluster.
+ A valid values is available after a call to allocate has been made
+ @return Current number of nodes in the cluster]]>
+      </doc>
+    </method>
+    <method name="getMatchingRequests" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="priority" type="org.apache.hadoop.yarn.api.records.Priority"/>
+      <param name="resourceName" type="java.lang.String"/>
+      <param name="capability" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <doc>
+      <![CDATA[Get outstanding <code>ContainerRequest</code>s matching the given
+ parameters. These ContainerRequests should have been added via
+ <code>addContainerRequest</code> earlier in the lifecycle. For performance,
+ the AMRMClient may return its internal collection directly without creating
+ a copy. Users should not perform mutable operations on the return value.
+ Each collection in the list contains requests with identical
+ <code>Resource</code> size that fit in the given capability. In a
+ collection, requests will be returned in the same order as they were added.
+ @return Collection of request matching the parameters]]>
+      </doc>
+    </method>
+    <method name="updateBlacklist"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="blacklistAdditions" type="java.util.List"/>
+      <param name="blacklistRemovals" type="java.util.List"/>
+      <doc>
+      <![CDATA[Update application's blacklist with addition or removal resources.
+
+ @param blacklistAdditions list of resources which should be added to the
+        application blacklist
+ @param blacklistRemovals list of resources which should be removed from the
+        application blacklist]]>
+      </doc>
+    </method>
+    <method name="setNMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nmTokenCache" type="org.apache.hadoop.yarn.client.api.NMTokenCache"/>
+      <doc>
+      <![CDATA[Set the NM token cache for the <code>AMRMClient</code>. This cache must
+ be shared with the {@link NMClient} used to manage containers for the
+ <code>AMRMClient</code>
+ <p>
+ If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
+ singleton instance will be used.
+
+ @param nmTokenCache the NM token cache to use.]]>
+      </doc>
+    </method>
+    <method name="getNMTokenCache" return="org.apache.hadoop.yarn.client.api.NMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the NM token cache of the <code>AMRMClient</code>. This cache must be
+ shared with the {@link NMClient} used to manage containers for the
+ <code>AMRMClient</code>.
+ <p>
+ If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
+ singleton instance will be used.
+
+ @return the NM token cache.]]>
+      </doc>
+    </method>
+    <method name="waitFor"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="check" type="com.google.common.base.Supplier"/>
+      <exception name="InterruptedException" type="java.lang.InterruptedException"/>
+      <doc>
+      <![CDATA[Wait for <code>check</code> to return true for each 1000 ms.
+ See also {@link #waitFor(com.google.common.base.Supplier, int)}
+ and {@link #waitFor(com.google.common.base.Supplier, int, int)}
+ @param check]]>
+      </doc>
+    </method>
+    <method name="waitFor"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="check" type="com.google.common.base.Supplier"/>
+      <param name="checkEveryMillis" type="int"/>
+      <exception name="InterruptedException" type="java.lang.InterruptedException"/>
+      <doc>
+      <![CDATA[Wait for <code>check</code> to return true for each
+ <code>checkEveryMillis</code> ms.
+ See also {@link #waitFor(com.google.common.base.Supplier, int, int)}
+ @param check user defined checker
+ @param checkEveryMillis interval to call <code>check</code>]]>
+      </doc>
+    </method>
+    <method name="waitFor"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="check" type="com.google.common.base.Supplier"/>
+      <param name="checkEveryMillis" type="int"/>
+      <param name="logInterval" type="int"/>
+      <exception name="InterruptedException" type="java.lang.InterruptedException"/>
+      <doc>
+      <![CDATA[Wait for <code>check</code> to return true for each
+ <code>checkEveryMillis</code> ms. In the main loop, this method will log
+ the message "waiting in main loop" for each <code>logInterval</code> times
+ iteration to confirm the thread is alive.
+ @param check user defined checker
+ @param checkEveryMillis interval to call <code>check</code>
+ @param logInterval interval to log for each]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.AMRMClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -->
+  <class name="AMRMClient.ContainerRequest" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AMRMClient.ContainerRequest" type="org.apache.hadoop.yarn.api.records.Resource, java.lang.String[], java.lang.String[], org.apache.hadoop.yarn.api.records.Priority"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Instantiates a {@link ContainerRequest} with the given constraints and
+ locality relaxation enabled.
+
+ @param capability
+          The {@link Resource} to be requested for each container.
+ @param nodes
+          Any hosts to request that the containers are placed on.
+ @param racks
+          Any racks to request that the containers are placed on. The
+          racks corresponding to any hosts requested will be automatically
+          added to this list.
+ @param priority
+          The priority at which to request the containers. Higher
+          priorities have lower numerical values.]]>
+      </doc>
+    </constructor>
+    <constructor name="AMRMClient.ContainerRequest" type="org.apache.hadoop.yarn.api.records.Resource, java.lang.String[], java.lang.String[], org.apache.hadoop.yarn.api.records.Priority, boolean"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Instantiates a {@link ContainerRequest} with the given constraints.
+
+ @param capability
+          The {@link Resource} to be requested for each container.
+ @param nodes
+          Any hosts to request that the containers are placed on.
+ @param racks
+          Any racks to request that the containers are placed on. The
+          racks corresponding to any hosts requested will be automatically
+          added to this list.
+ @param priority
+          The priority at which to request the containers. Higher
+          priorities have lower numerical values.
+ @param relaxLocality
+          If true, containers for this request may be assigned on hosts
+          and racks other than the ones explicitly requested.]]>
+      </doc>
+    </constructor>
+    <constructor name="AMRMClient.ContainerRequest" type="org.apache.hadoop.yarn.api.records.Resource, java.lang.String[], java.lang.String[], org.apache.hadoop.yarn.api.records.Priority, boolean, java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Instantiates a {@link ContainerRequest} with the given constraints.
+
+ @param capability
+          The {@link Resource} to be requested for each container.
+ @param nodes
+          Any hosts to request that the containers are placed on.
+ @param racks
+          Any racks to request that the containers are placed on. The
+          racks corresponding to any hosts requested will be automatically
+          added to this list.
+ @param priority
+          The priority at which to request the containers. Higher
+          priorities have lower numerical values.
+ @param relaxLocality
+          If true, containers for this request may be assigned on hosts
+          and racks other than the ones explicitly requested.
+ @param nodeLabelsExpression
+          Set node labels to allocate resource, now we only support
+          asking for only a single node label]]>
+      </doc>
+    </constructor>
+    <method name="getCapability" return="org.apache.hadoop.yarn.api.records.Resource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getNodes" return="java.util.List"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getRacks" return="java.util.List"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getPriority" return="org.apache.hadoop.yarn.api.records.Priority"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getRelaxLocality" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getNodeLabelExpression" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <doc>
+    <![CDATA[Object to represent a single container request for resources. Scheduler
+ documentation should be consulted for the specifics of how the parameters
+ are honored.
+
+ By default, YARN schedulers try to allocate containers at the requested
+ locations but they may relax the constraints in order to expedite meeting
+ allocations limits. They first relax the constraint to the same rack as the
+ requested node and then to anywhere in the cluster. The relaxLocality flag
+ may be used to disable locality relaxation and request containers at only
+ specific locations. The following conditions apply.
+ <ul>
+ <li>Within a priority, all container requests must have the same value for
+ locality relaxation. Either enabled or disabled.</li>
+ <li>If locality relaxation is disabled, then across requests, locations at
+ different network levels may not be specified. E.g. its invalid to make a
+ request for a specific node and another request for a specific rack.</li>
+ <li>If locality relaxation is disabled, then only within the same request,
+ a node and its rack may be specified together. This allows for a specific
+ rack with a preference for a specific node within that rack.</li>
+ <li></li>
+ </ul>
+ To re-enable locality relaxation at a given priority, all pending requests
+ with locality relaxation disabled must be first removed. Then they can be
+ added back with locality relaxation enabled.
+
+ All getters return immutable values.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -->
+  <!-- start class org.apache.hadoop.yarn.client.api.InvalidContainerRequestException -->
+  <class name="InvalidContainerRequestException" extends="org.apache.hadoop.yarn.exceptions.YarnRuntimeException"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="InvalidContainerRequestException" type="java.lang.Throwable"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="InvalidContainerRequestException" type="java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="InvalidContainerRequestException" type="java.lang.String, java.lang.Throwable"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <doc>
+    <![CDATA[Thrown when an arguments are combined to construct a
+ <code>AMRMClient.ContainerRequest</code> in an invalid way.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.InvalidContainerRequestException -->
+  <!-- start class org.apache.hadoop.yarn.client.api.NMClient -->
+  <class name="NMClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NMClient" type="java.lang.String"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createNMClient" return="org.apache.hadoop.yarn.client.api.NMClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new instance of NMClient.]]>
+      </doc>
+    </method>
+    <method name="createNMClient" return="org.apache.hadoop.yarn.client.api.NMClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Create a new instance of NMClient.]]>
+      </doc>
+    </method>
+    <method name="startContainer" return="java.util.Map"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="container" type="org.apache.hadoop.yarn.api.records.Container"/>
+      <param name="containerLaunchContext" type="org.apache.hadoop.yarn.api.records.ContainerLaunchContext"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Start an allocated container.</p>
+
+ <p>The <code>ApplicationMaster</code> or other applications that use the
+ client must provide the details of the allocated container, including the
+ Id, the assigned node's Id and the token via {@link Container}. In
+ addition, the AM needs to provide the {@link ContainerLaunchContext} as
+ well.</p>
+
+ @param container the allocated container
+ @param containerLaunchContext the context information needed by the
+                               <code>NodeManager</code> to launch the
+                               container
+ @return a map between the auxiliary service names and their outputs
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="stopContainer"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Stop an started container.</p>
+
+ @param containerId the Id of the started container
+ @param nodeId the Id of the <code>NodeManager</code>
+
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainerStatus" return="org.apache.hadoop.yarn.api.records.ContainerStatus"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Query the status of a container.</p>
+
+ @param containerId the Id of the started container
+ @param nodeId the Id of the <code>NodeManager</code>
+
+ @return the status of a container
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="cleanupRunningContainersOnStop"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="enabled" type="boolean"/>
+      <doc>
+      <![CDATA[<p>Set whether the containers that are started by this client, and are
+ still running should be stopped when the client stops. By default, the
+ feature should be enabled.</p> However, containers will be stopped only
+ when service is stopped. i.e. after {@link NMClient#stop()}.
+
+ @param enabled whether the feature is enabled or not]]>
+      </doc>
+    </method>
+    <method name="setNMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nmTokenCache" type="org.apache.hadoop.yarn.client.api.NMTokenCache"/>
+      <doc>
+      <![CDATA[Set the NM Token cache of the <code>NMClient</code>. This cache must be
+ shared with the {@link AMRMClient} that requested the containers managed
+ by this <code>NMClient</code>
+ <p>
+ If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
+ singleton instance will be used.
+
+ @param nmTokenCache the NM token cache to use.]]>
+      </doc>
+    </method>
+    <method name="getNMTokenCache" return="org.apache.hadoop.yarn.client.api.NMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the NM token cache of the <code>NMClient</code>. This cache must be
+ shared with the {@link AMRMClient} that requested the containers managed
+ by this <code>NMClient</code>
+ <p>
+ If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
+ singleton instance will be used.
+
+ @return the NM token cache]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.NMClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.NMTokenCache -->
+  <class name="NMTokenCache" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NMTokenCache"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Creates a NM token cache instance.]]>
+      </doc>
+    </constructor>
+    <method name="getSingleton" return="org.apache.hadoop.yarn.client.api.NMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Returns the singleton NM token cache.
+
+ @return the singleton NM token cache.]]>
+      </doc>
+    </method>
+    <method name="getNMToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeAddr" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Returns NMToken, null if absent. Only the singleton obtained from
+ {@link #getSingleton()} is looked at for the tokens. If you are using your
+ own NMTokenCache that is different from the singleton, use
+ {@link #getToken(String) }
+
+ @param nodeAddr
+ @return {@link Token} NMToken required for communicating with node manager]]>
+      </doc>
+    </method>
+    <method name="setNMToken"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeAddr" type="java.lang.String"/>
+      <param name="token" type="org.apache.hadoop.yarn.api.records.Token"/>
+      <doc>
+      <![CDATA[Sets the NMToken for node address only in the singleton obtained from
+ {@link #getSingleton()}. If you are using your own NMTokenCache that is
+ different from the singleton, use {@link #setToken(String, Token) }
+
+ @param nodeAddr
+          node address (host:port)
+ @param token
+          NMToken]]>
+      </doc>
+    </method>
+    <method name="getToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeAddr" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Returns NMToken, null if absent
+ @param nodeAddr
+ @return {@link Token} NMToken required for communicating with node
+         manager]]>
+      </doc>
+    </method>
+    <method name="setToken"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeAddr" type="java.lang.String"/>
+      <param name="token" type="org.apache.hadoop.yarn.api.records.Token"/>
+      <doc>
+      <![CDATA[Sets the NMToken for node address
+ @param nodeAddr node address (host:port)
+ @param token NMToken]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[NMTokenCache manages NMTokens required for an Application Master
+ communicating with individual NodeManagers.
+ <p>
+ By default Yarn client libraries {@link AMRMClient} and {@link NMClient} use
+ {@link #getSingleton()} instance of the cache.
+ <ul>
+   <li>
+     Using the singleton instance of the cache is appropriate when running a
+     single ApplicationMaster in the same JVM.
+   </li>
+   <li>
+     When using the singleton, users don't need to do anything special,
+     {@link AMRMClient} and {@link NMClient} are already set up to use the
+     default singleton {@link NMTokenCache}
+     </li>
+ </ul>
+ If running multiple Application Masters in the same JVM, a different cache
+ instance should be used for each Application Master.
+ <ul>
+   <li>
+     If using the {@link AMRMClient} and the {@link NMClient}, setting up
+     and using an instance cache is as follows:
+ <pre>
+   NMTokenCache nmTokenCache = new NMTokenCache();
+   AMRMClient rmClient = AMRMClient.createAMRMClient();
+   NMClient nmClient = NMClient.createNMClient();
+   nmClient.setNMTokenCache(nmTokenCache);
+   ...
+ </pre>
+   </li>
+   <li>
+     If using the {@link AMRMClientAsync} and the {@link NMClientAsync},
+     setting up and using an instance cache is as follows:
+ <pre>
+   NMTokenCache nmTokenCache = new NMTokenCache();
+   AMRMClient rmClient = AMRMClient.createAMRMClient();
+   NMClient nmClient = NMClient.createNMClient();
+   nmClient.setNMTokenCache(nmTokenCache);
+   AMRMClientAsync rmClientAsync = new AMRMClientAsync(rmClient, 1000, [AMRM_CALLBACK]);
+   NMClientAsync nmClientAsync = new NMClientAsync("nmClient", nmClient, [NM_CALLBACK]);
+   ...
+ </pre>
+   </li>
+   <li>
+     If using {@link ApplicationMasterProtocol} and
+     {@link ContainerManagementProtocol} directly, setting up and using an
+     instance cache is as follows:
+ <pre>
+   NMTokenCache nmTokenCache = new NMTokenCache();
+   ...
+   ApplicationMasterProtocol amPro = ClientRMProxy.createRMProxy(conf, ApplicationMasterProtocol.class);
+   ...
+   AllocateRequest allocateRequest = ...
+   ...
+   AllocateResponse allocateResponse = rmClient.allocate(allocateRequest);
+   for (NMToken token : allocateResponse.getNMTokens()) {
+     nmTokenCache.setToken(token.getNodeId().toString(), token.getToken());
+   }
+   ...
+   ContainerManagementProtocolProxy nmPro = ContainerManagementProtocolProxy(conf, nmTokenCache);
+   ...
+   nmPro.startContainer(container, containerContext);
+   ...
+ </pre>
+   </li>
+ </ul>
+ It is also possible to mix the usage of a client ({@code AMRMClient} or
+ {@code NMClient}, or the async versions of them) with a protocol proxy
+ ({@code ContainerManagementProtocolProxy} or
+ {@code ApplicationMasterProtocol}).]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.NMTokenCache -->
+  <!-- start class org.apache.hadoop.yarn.client.api.SharedCacheClient -->
+  <class name="SharedCacheClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="SharedCacheClient" type="java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createSharedCacheClient" return="org.apache.hadoop.yarn.client.api.SharedCacheClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="use" return="org.apache.hadoop.fs.Path"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="resourceKey" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<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]]>
+      </doc>
+    </method>
+    <method name="release"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="resourceKey" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<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]]>
+      </doc>
+    </method>
+    <method name="getFileChecksum" return="java.lang.String"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="sourceFile" type="org.apache.hadoop.fs.Path"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[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]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[This is the client for YARN's shared cache.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.SharedCacheClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.YarnClient -->
+  <class name="YarnClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="YarnClient" type="java.lang.String"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createYarnClient" return="org.apache.hadoop.yarn.client.api.YarnClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new instance of YarnClient.]]>
+      </doc>
+    </method>
+    <method name="createApplication" return="org.apache.hadoop.yarn.client.api.YarnClientApplication"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Obtain a {@link YarnClientApplication} for a new application,
+ which in turn contains the {@link ApplicationSubmissionContext} and
+ {@link org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse}
+ objects.
+ </p>
+
+ @return {@link YarnClientApplication} built for a new application
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="submitApplication" return="org.apache.hadoop.yarn.api.records.ApplicationId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appContext" type="org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Submit a new application to <code>YARN.</code> It is a blocking call - it
+ will not return {@link ApplicationId} until the submitted application is
+ submitted successfully and accepted by the ResourceManager.
+ </p>
+
+ <p>
+ Users should provide an {@link ApplicationId} as part of the parameter
+ {@link ApplicationSubmissionContext} when submitting a new application,
+ otherwise it will throw the {@link ApplicationIdNotProvidedException}.
+ </p>
+
+ <p>This internally calls {@link ApplicationClientProtocol#submitApplication
+ (SubmitApplicationRequest)}, and after that, it internally invokes
+ {@link ApplicationClientProtocol#getApplicationReport
+ (GetApplicationReportRequest)} and waits till it can make sure that the
+ application gets properly submitted. If RM fails over or RM restart
+ happens before ResourceManager saves the application's state,
+ {@link ApplicationClientProtocol
+ #getApplicationReport(GetApplicationReportRequest)} will throw
+ the {@link ApplicationNotFoundException}. This API automatically resubmits
+ the application with the same {@link ApplicationSubmissionContext} when it
+ catches the {@link ApplicationNotFoundException}</p>
+
+ @param appContext
+          {@link ApplicationSubmissionContext} containing all the details
+          needed to submit a new application
+ @return {@link ApplicationId} of the accepted application
+ @throws YarnException
+ @throws IOException
+ @see #createApplication()]]>
+      </doc>
+    </method>
+    <method name="killApplication"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Kill an application identified by given ID.
+ </p>
+
+ @param applicationId
+          {@link ApplicationId} of the application that needs to be killed
+ @throws YarnException
+           in case of errors or if YARN rejects the request due to
+           access-control restrictions.
+ @throws IOException
+ @see #getQueueAclsInfo()]]>
+      </doc>
+    </method>
+    <method name="getApplicationReport" return="org.apache.hadoop.yarn.api.records.ApplicationReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given Application.
+ </p>
+
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access then the following
+ fields in the report will be set to stubbed values:
+ <ul>
+ <li>host - set to "N/A"</li>
+ <li>RPC port - set to -1</li>
+ <li>client token - set to "N/A"</li>
+ <li>diagnostics - set to "N/A"</li>
+ <li>tracking URL - set to "N/A"</li>
+ <li>original tracking URL - set to "N/A"</li>
+ <li>resource usage report - all values are -1</li>
+ </ul>
+
+ @param appId
+          {@link ApplicationId} of the application that needs a report
+ @return application report
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getAMRMToken" return="org.apache.hadoop.security.token.Token"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Get the AMRM token of the application.
+ <p>
+ The AMRM token is required for AM to RM scheduling operations. For
+ managed Application Masters Yarn takes care of injecting it. For unmanaged
+ Applications Masters, the token must be obtained via this method and set
+ in the {@link org.apache.hadoop.security.UserGroupInformation} of the
+ current user.
+ <p>
+ The AMRM token will be returned only if all the following conditions are
+ met:
+ <ul>
+   <li>the requester is the owner of the ApplicationMaster</li>
+   <li>the application master is an unmanaged ApplicationMaster</li>
+   <li>the application master is in ACCEPTED state</li>
+ </ul>
+ Else this method returns NULL.
+
+ @param appId {@link ApplicationId} of the application to get the AMRM token
+ @return the AMRM token if available
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of all Applications in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @return a list of reports of all running applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationTypes" type="java.util.Set"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of Applications
+ matching the given application types in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @param applicationTypes
+ @return a list of reports of applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationStates" type="java.util.EnumSet"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of Applications matching the given
+ application states in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @param applicationStates
+ @return a list of reports of applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationTypes" type="java.util.Set"/>
+      <param name="applicationStates" type="java.util.EnumSet"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of Applications matching the given
+ application types and application states in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @param applicationTypes
+ @param applicationStates
+ @return a list of reports of applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getYarnClusterMetrics" return="org.apache.hadoop.yarn.api.records.YarnClusterMetrics"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get metrics ({@link YarnClusterMetrics}) about the cluster.
+ </p>
+
+ @return cluster metrics
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getNodeReports" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="states" type="org.apache.hadoop.yarn.api.records.NodeState[]"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of nodes ({@link NodeReport}) in the cluster.
+ </p>
+
+ @param states The {@link NodeState}s to filter on. If no filter states are
+          given, nodes in all states will be returned.
+ @return A list of node reports
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getRMDelegationToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="renewer" type="org.apache.hadoop.io.Text"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a delegation token so as to be able to talk to YARN using those tokens.
+
+ @param renewer
+          Address of the renewer who can renew these tokens when needed by
+          securely talking to YARN.
+ @return a delegation token ({@link Token}) that can be used to
+         talk to YARN
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getQueueInfo" return="org.apache.hadoop.yarn.api.records.QueueInfo"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="queueName" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information ({@link QueueInfo}) about a given <em>queue</em>.
+ </p>
+
+ @param queueName
+          Name of the queue whose information is needed
+ @return queue information
+ @throws YarnException
+           in case of errors or if YARN rejects the request due to
+           access-control restrictions.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getAllQueues" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information ({@link QueueInfo}) about all queues, recursively if there
+ is a hierarchy
+ </p>
+
+ @return a list of queue-information for all queues
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getRootQueueInfos" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information ({@link QueueInfo}) about top level queues.
+ </p>
+
+ @return a list of queue-information for all the top-level queues
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getChildQueueInfos" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="parent" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information ({@link QueueInfo}) about all the immediate children queues
+ of the given queue
+ </p>
+
+ @param parent
+          Name of the queue whose child-queues' information is needed
+ @return a list of queue-information for all queues who are direct children
+         of the given parent queue.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getQueueAclsInfo" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information about <em>acls</em> for <em>current user</em> on all the
+ existing queues.
+ </p>
+
+ @return a list of queue acls ({@link QueueUserACLInfo}) for
+         <em>current user</em>
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttemptReport" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given ApplicationAttempt.
+ </p>
+
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+
+ @param applicationAttemptId
+          {@link ApplicationAttemptId} of the application attempt that needs
+          a report
+ @return application attempt report
+ @throws YarnException
+ @throws ApplicationAttemptNotFoundException if application attempt
+         not found
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttempts" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of all (ApplicationAttempts) of Application in the cluster.
+ </p>
+
+ @param applicationId
+ @return a list of reports for all application attempts for specified
+         application.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainerReport" return="org.apache.hadoop.yarn.api.records.ContainerReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given Container.
+ </p>
+
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+
+ @param containerId
+          {@link ContainerId} of the container that needs a report
+ @return container report
+ @throws YarnException
+ @throws ContainerNotFoundException if container not found.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainers" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of all (Containers) of ApplicationAttempt in the cluster.
+ </p>
+
+ @param applicationAttemptId
+ @return a list of reports of all containers for specified application
+         attempts
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="moveApplicationAcrossQueues"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="queue" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Attempts to move the given application to the given queue.
+ </p>
+
+ @param appId
+    Application to move.
+ @param queue
+    Queue to place it in to.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="submitReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to submit a new reservation to the
+ {@code ResourceManager}.
+ </p>
+
+ <p>
+ The client packages all details of its request in a
+ {@link ReservationSubmissionRequest} object. This contains information
+ about the amount of capacity, temporal constraints, and gang needs.
+ Furthermore, the reservation might be composed of multiple stages, with
+ ordering dependencies among them.
+ </p>
+
+ <p>
+ In order to respond, a new admission control component in the
+ {@code ResourceManager} performs an analysis of the resources that have
+ been committed over the period of time the user is requesting, verify that
+ the user requests can be fulfilled, and that it respect a sharing policy
+ (e.g., {@code CapacityOverTimePolicy}). Once it has positively determined
+ that the ReservationRequest is satisfiable the {@code ResourceManager}
+ answers with a {@link ReservationSubmissionResponse} that includes a
+ {@link ReservationId}. Upon failure to find a valid allocation the response
+ is an exception with the message detailing the reason of failure.
+ </p>
+
+ <p>
+ The semantics guarantees that the {@link ReservationId} returned,
+ corresponds to a valid reservation existing in the time-range request by
+ the user. The amount of capacity dedicated to such reservation can vary
+ overtime, depending of the allocation that has been determined. But it is
+ guaranteed to satisfy all the constraint expressed by the user in the
+ {@link ReservationDefinition}
+ </p>
+
+ @param request request to submit a new Reservation
+ @return response contains the {@link ReservationId} on accepting the
+         submission
+ @throws YarnException if the reservation cannot be created successfully
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="updateReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to update an existing Reservation. This is
+ referred to as a re-negotiation process, in which a user that has
+ previously submitted a Reservation.
+ </p>
+
+ <p>
+ The allocation is attempted by virtually substituting all previous
+ allocations related to this Reservation with new ones, that satisfy the new
+ {@link ReservationDefinition}. Upon success the previous allocation is
+ atomically substituted by the new one, and on failure (i.e., if the system
+ cannot find a valid allocation for the updated request), the previous
+ allocation remains valid.
+ </p>
+
+ @param request to update an existing Reservation (the
+          {@link ReservationUpdateRequest} should refer to an existing valid
+          {@link ReservationId})
+ @return response empty on successfully updating the existing reservation
+ @throws YarnException if the request is invalid or reservation cannot be
+           updated successfully
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="deleteReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to remove an existing Reservation.
+ </p>
+
+ @param request to remove an existing Reservation (the
+          {@link ReservationDeleteRequest} should refer to an existing valid
+          {@link ReservationId})
+ @return response empty on successfully deleting the existing reservation
+ @throws YarnException if the request is invalid or reservation cannot be
+           deleted successfully
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getNodeToLabels" return="java.util.Map"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by client to get node to labels mappings in existing cluster
+ </p>
+
+ @return node to labels mappings
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getLabelsToNodes" return="java.util.Map"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by client to get labels to nodes mapping
+ in existing cluster
+ </p>
+
+ @return node to labels mappings
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getLabelsToNodes" return="java.util.Map"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="labels" type="java.util.Set"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by client to get labels to nodes mapping
+ for specified labels in existing cluster
+ </p>
+
+ @param labels labels for which labels to nodes mapping has to be retrieved
+ @return labels to nodes mappings for specific labels
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getClusterNodeLabels" return="java.util.Set"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by client to get node labels in the cluster
+ </p>
+
+ @return cluster node labels collection
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.YarnClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.YarnClientApplication -->
+  <class name="YarnClientApplication" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="YarnClientApplication" type="org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse, org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getNewApplicationResponse" return="org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getApplicationSubmissionContext" return="org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.YarnClientApplication -->
+</package>
+<package name="org.apache.hadoop.yarn.client.api.async">
+  <!-- start class org.apache.hadoop.yarn.client.api.async.AMRMClientAsync -->
+  <class name="AMRMClientAsync" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AMRMClientAsync" type="int, org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AMRMClientAsync" type="org.apache.hadoop.yarn.client.api.AMRMClient, int, org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createAMRMClientAsync" return="org.apache.hadoop.yarn.client.api.async.AMRMClientAsync"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="intervalMs" type="int"/>
+      <param name="callbackHandler" type="org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler"/>
+    </method>
+    <method name="createAMRMClientAsync" return="org.apache.hadoop.yarn.client.api.async.AMRMClientAsync"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="client" type="org.apache.hadoop.yarn.client.api.AMRMClient"/>
+      <param name="intervalMs" type="int"/>
+      <param name="callbackHandler" type="org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler"/>
+    </method>
+    <method name="setHeartbeatInterval"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="interval" type="int"/>
+    </method>
+    <method name="getMatchingRequests" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="priority" type="org.apache.hadoop.yarn.api.records.Priority"/>
+      <param name="resourceName" type="java.lang.String"/>
+      <param name="capability" type="org.apache.hadoop.yarn.api.records.Resource"/>
+    </method>
+    <method name="registerApplicationMaster" return="org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appHostName" type="java.lang.String"/>
+      <param name="appHostPort" type="int"/>
+      <param name="appTrackingUrl" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Registers this application master with the resource manager. On successful
+ registration, starts the heartbeating thread.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="unregisterApplicationMaster"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appStatus" type="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"/>
+      <param name="appMessage" type="java.lang.String"/>
+      <param name="appTrackingUrl" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Unregister the application master. This must be called in the end.
+ @param appStatus Success/Failure status of the master
+ @param appMessage Diagnostics message on failure
+ @param appTrackingUrl New URL to get master info
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="addContainerRequest"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="T"/>
+      <doc>
+      <![CDATA[Request containers for resources before calling <code>allocate</code>
+ @param req Resource request]]>
+      </doc>
+    </method>
+    <method name="removeContainerRequest"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="T"/>
+      <doc>
+      <![CDATA[Remove previous container request. The previous container request may have
+ already been sent to the ResourceManager. So even after the remove request
+ the app must be prepared to receive an allocation for the previous request
+ even after the remove request
+ @param req Resource request]]>
+      </doc>
+    </method>
+    <method name="releaseAssignedContainer"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <doc>
+      <![CDATA[Release containers assigned by the Resource Manager. If the app cannot use
+ the container or wants to give up the container then it can release them.
+ The app needs to make new requests for the released resource capability if
+ it still needs it. eg. it released non-local resources
+ @param containerId]]>
+      </doc>
+    </method>
+    <method name="getAvailableResources" return="org.apache.hadoop.yarn.api.records.Resource"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the currently available resources in the cluster.
+ A valid value is available after a call to allocate has been made
+ @return Currently available resources]]>
+      </doc>
+    </method>
+    <method name="getClusterNodeCount" return="int"
+      abstract="true" native="false" synchronized="false"
+      

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[06/51] [abbrv] hadoop git commit: YARN-5176. More test cases for queuing of containers at the NM. (Konstantinos Karanasos via asuresh)

Posted by vv...@apache.org.
YARN-5176. More test cases for queuing of containers at the NM. (Konstantinos Karanasos via asuresh)


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

Branch: refs/heads/YARN-3926
Commit: 76f0800c21f49fba01694cbdc870103053da802c
Parents: 58be55b
Author: Arun Suresh <as...@apache.org>
Authored: Tue Jun 7 17:16:18 2016 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Tue Jun 7 17:16:18 2016 -0700

----------------------------------------------------------------------
 .../queuing/QueuingContainerManagerImpl.java    |  11 +
 .../BaseContainerManagerTest.java               |  64 +++
 .../containermanager/TestContainerManager.java  |  76 +---
 .../queuing/TestQueuingContainerManager.java    | 388 +++++++++++++++----
 4 files changed, 391 insertions(+), 148 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/76f0800c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java
index 1ce3356..a1e3bdb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java
@@ -160,6 +160,7 @@ public class QueuingContainerManagerImpl extends ContainerManagerImpl {
           containerTokenId.getExecutionType());
 
       if (foundInQueue) {
+        LOG.info("Removing queued container with ID " + containerID);
         this.context.getQueuingContext().getKilledQueuedContainers().put(
             containerTokenId,
             "Queued container request removed by ApplicationMaster.");
@@ -502,6 +503,16 @@ public class QueuingContainerManagerImpl extends ContainerManagerImpl {
     return allocatedOpportunisticContainers.size();
   }
 
+  @VisibleForTesting
+  public int getNumQueuedGuaranteedContainers() {
+    return queuedGuaranteedContainers.size();
+  }
+
+  @VisibleForTesting
+  public int getNumQueuedOpportunisticContainers() {
+    return queuedOpportunisticContainers.size();
+  }
+
   class QueuingApplicationEventDispatcher implements
       EventHandler<ApplicationEvent> {
     private EventHandler<ApplicationEvent> applicationEventDispatcher;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76f0800c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
index ab60288..4f0e5c3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
@@ -40,10 +40,17 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -51,6 +58,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
+import org.apache.hadoop.yarn.server.api.ContainerType;
 import org.apache.hadoop.yarn.server.api.ResourceTracker;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
@@ -71,6 +79,7 @@ import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreServic
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.After;
 import org.junit.Before;
 
@@ -354,4 +363,59 @@ public abstract class BaseContainerManagerTest {
     Assert.assertEquals("ContainerState is not correct (timedout)",
         finalState, currentState);
   }
+
+  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
+      NodeId nodeId, String user,
+      NMContainerTokenSecretManager containerTokenSecretManager)
+      throws IOException {
+    return createContainerToken(cId, rmIdentifier, nodeId, user,
+      containerTokenSecretManager, null);
+  }
+
+  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
+      NodeId nodeId, String user,
+      NMContainerTokenSecretManager containerTokenSecretManager,
+      LogAggregationContext logAggregationContext)
+      throws IOException {
+    Resource r = BuilderUtils.newResource(1024, 1);
+    return createContainerToken(cId, rmIdentifier, nodeId, user, r,
+        containerTokenSecretManager, logAggregationContext);
+  }
+
+  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
+      NodeId nodeId, String user, Resource resource,
+      NMContainerTokenSecretManager containerTokenSecretManager,
+      LogAggregationContext logAggregationContext)
+      throws IOException {
+    ContainerTokenIdentifier containerTokenIdentifier =
+        new ContainerTokenIdentifier(cId, nodeId.toString(), user, resource,
+          System.currentTimeMillis() + 100000L, 123, rmIdentifier,
+          Priority.newInstance(0), 0, logAggregationContext, null);
+    return BuilderUtils.newContainerToken(nodeId, containerTokenSecretManager
+        .retrievePassword(containerTokenIdentifier),
+            containerTokenIdentifier);
+  }
+
+  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
+      NodeId nodeId, String user, Resource resource,
+      NMContainerTokenSecretManager containerTokenSecretManager,
+      LogAggregationContext logAggregationContext, ExecutionType executionType)
+      throws IOException {
+    ContainerTokenIdentifier containerTokenIdentifier =
+        new ContainerTokenIdentifier(cId, nodeId.toString(), user, resource,
+            System.currentTimeMillis() + 100000L, 123, rmIdentifier,
+            Priority.newInstance(0), 0, logAggregationContext, null,
+            ContainerType.TASK, executionType);
+    return BuilderUtils.newContainerToken(nodeId, containerTokenSecretManager
+            .retrievePassword(containerTokenIdentifier),
+        containerTokenIdentifier);
+  }
+
+  public static ContainerId createContainerId(int id) {
+    ApplicationId appId = ApplicationId.newInstance(0, 0);
+    ApplicationAttemptId appAttemptId =
+        ApplicationAttemptId.newInstance(appId, 1);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, id);
+    return containerId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76f0800c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index 702198e..10b9155 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -18,6 +18,11 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager;
 
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.timeout;
+import static org.mockito.Mockito.verify;
+
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileReader;
@@ -38,10 +43,10 @@ import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.util.Shell;
-import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
@@ -58,13 +63,9 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
-import org.apache.hadoop.yarn.api.records.LogAggregationContext;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.SignalContainerCommand;
@@ -75,7 +76,6 @@ import org.apache.hadoop.yarn.exceptions.InvalidContainerException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
-import org.apache.hadoop.yarn.server.api.ContainerType;
 import org.apache.hadoop.yarn.server.api.ResourceManagerConstants;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrDecreaseContainersResourceEvent;
@@ -90,8 +90,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.Conta
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext;
-import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
-import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.Assert;
 import org.junit.Before;
@@ -99,11 +97,6 @@ import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 import org.mockito.Mockito;
 
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.timeout;
-import static org.mockito.Mockito.verify;
-
 public class TestContainerManager extends BaseContainerManagerTest {
 
   public TestContainerManager() throws UnsupportedFileSystemException {
@@ -119,14 +112,6 @@ public class TestContainerManager extends BaseContainerManagerTest {
   public void setup() throws IOException {
     super.setup();
   }
-
-  public static ContainerId createContainerId(int id) {
-    ApplicationId appId = ApplicationId.newInstance(0, 0);
-    ApplicationAttemptId appAttemptId =
-        ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId containerId = ContainerId.newContainerId(appAttemptId, id);
-    return containerId;
-  }
   
   @Override
   protected ContainerManagerImpl
@@ -1153,53 +1138,6 @@ public class TestContainerManager extends BaseContainerManagerTest {
     assertEquals(targetResource, containerStatus.getCapability());
   }
 
-  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
-      NodeId nodeId, String user,
-      NMContainerTokenSecretManager containerTokenSecretManager)
-      throws IOException {
-    return createContainerToken(cId, rmIdentifier, nodeId, user,
-      containerTokenSecretManager, null);
-  }
-
-  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
-      NodeId nodeId, String user,
-      NMContainerTokenSecretManager containerTokenSecretManager,
-      LogAggregationContext logAggregationContext)
-      throws IOException {
-    Resource r = BuilderUtils.newResource(1024, 1);
-    return createContainerToken(cId, rmIdentifier, nodeId, user, r,
-        containerTokenSecretManager, logAggregationContext);
-  }
-
-  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
-      NodeId nodeId, String user, Resource resource,
-      NMContainerTokenSecretManager containerTokenSecretManager,
-      LogAggregationContext logAggregationContext)
-      throws IOException {
-    ContainerTokenIdentifier containerTokenIdentifier =
-        new ContainerTokenIdentifier(cId, nodeId.toString(), user, resource,
-          System.currentTimeMillis() + 100000L, 123, rmIdentifier,
-          Priority.newInstance(0), 0, logAggregationContext, null);
-    return BuilderUtils.newContainerToken(nodeId, containerTokenSecretManager
-        .retrievePassword(containerTokenIdentifier),
-            containerTokenIdentifier);
-  }
-
-  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
-      NodeId nodeId, String user, Resource resource,
-      NMContainerTokenSecretManager containerTokenSecretManager,
-      LogAggregationContext logAggregationContext, ExecutionType executionType)
-      throws IOException {
-    ContainerTokenIdentifier containerTokenIdentifier =
-        new ContainerTokenIdentifier(cId, nodeId.toString(), user, resource,
-            System.currentTimeMillis() + 100000L, 123, rmIdentifier,
-            Priority.newInstance(0), 0, logAggregationContext, null,
-            ContainerType.TASK, executionType);
-    return BuilderUtils.newContainerToken(nodeId, containerTokenSecretManager
-            .retrievePassword(containerTokenIdentifier),
-        containerTokenIdentifier);
-  }
-
   @Test
   public void testOutputThreadDumpSignal() throws IOException,
       InterruptedException, YarnException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76f0800c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java
index 8a0d155..4d44d8d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java
@@ -18,6 +18,11 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.queuing;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -32,42 +37,27 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.LocalResourceType;
-import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
-import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerManager;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
-
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
-
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor
-    .ContainersMonitorImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.MockResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.MockResourceCalculatorProcessTree;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
-
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class TestQueuingContainerManager extends TestContainerManager {
+/**
+ * Class for testing the {@link QueuingContainerManagerImpl}.
+ */
+public class TestQueuingContainerManager extends BaseContainerManagerTest {
 
   interface HasResources {
     boolean decide(Context context, ContainerId cId);
@@ -120,15 +110,6 @@ public class TestQueuingContainerManager extends TestContainerManager {
       }
 
       @Override
-      protected void authorizeGetAndStopContainerRequest(
-          ContainerId containerId, Container container, boolean stopRequest,
-          NMTokenIdentifier identifier) throws YarnException {
-        if (container == null || container.getUser().equals("Fail")) {
-          throw new YarnException("Reject this container");
-        }
-      }
-
-      @Override
       protected ContainersMonitor createContainersMonitor(
           ContainerExecutor exec) {
         return new ContainersMonitorImpl(exec, dispatcher, this.context) {
@@ -148,7 +129,7 @@ public class TestQueuingContainerManager extends TestContainerManager {
 
           @Override
           public long getVCoresAllocatedForContainers() {
-            return 2;
+            return 4;
           }
         };
       }
@@ -186,54 +167,17 @@ public class TestQueuingContainerManager extends TestContainerManager {
   }
 
   /**
-   * Test to verify that an OPPORTUNISTIC container is killed when
-   * a GUARANTEED container arrives and all the Node Resources are used up
-   *
-   * For this specific test case, 4 containers are requested (last one being
-   * guaranteed). Assumptions :
-   * 1) The first OPPORTUNISTIC Container will start running
-   * 2) The second and third OPP containers will be queued
-   * 3) When the GUARANTEED container comes in, the running OPP container
-   *    will be killed to make room
-   * 4) After the GUARANTEED container finishes, the remaining 2 OPP
-   *    containers will be dequeued and run.
-   * 5) Only the first OPP container will be killed.
-   *
+   * Starting one GUARANTEED and one OPPORTUNISTIC container.
    * @throws Exception
    */
   @Test
-  public void testSimpleOpportunisticContainer() throws Exception {
+  public void testStartMultipleContainers() throws Exception {
     shouldDeleteWait = true;
     containerManager.start();
 
-    // ////// Create the resources for the container
-    File dir = new File(tmpDir, "dir");
-    dir.mkdirs();
-    File file = new File(dir, "file");
-    PrintWriter fileWriter = new PrintWriter(file);
-    fileWriter.write("Hello World!");
-    fileWriter.close();
-
-    // ////// Construct the container-spec.
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
-    URL resource_alpha =
-        ConverterUtils.getYarnUrlFromPath(localFS
-            .makeQualified(new Path(file.getAbsolutePath())));
-    LocalResource rsrc_alpha =
-        recordFactory.newRecordInstance(LocalResource.class);
-    rsrc_alpha.setResource(resource_alpha);
-    rsrc_alpha.setSize(-1);
-    rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
-    rsrc_alpha.setType(LocalResourceType.FILE);
-    rsrc_alpha.setTimestamp(file.lastModified());
-    String destinationFile = "dest_file";
-    Map<String, LocalResource> localResources =
-        new HashMap<String, LocalResource>();
-    localResources.put(destinationFile, rsrc_alpha);
-    containerLaunchContext.setLocalResources(localResources);
-
-    // Start 3 OPPORTUNISTIC containers and 1 GUARANTEED container
+
     List<StartContainerRequest> list = new ArrayList<>();
     list.add(StartContainerRequest.newInstance(
         containerLaunchContext,
@@ -241,6 +185,122 @@ public class TestQueuingContainerManager extends TestContainerManager {
             context.getNodeId(),
             user, BuilderUtils.newResource(1024, 1),
             context.getContainerTokenSecretManager(), null,
+            ExecutionType.GUARANTEED)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(1024, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+
+    BaseContainerManagerTest.waitForContainerState(containerManager,
+        createContainerId(0),
+        org.apache.hadoop.yarn.api.records.ContainerState.RUNNING);
+    BaseContainerManagerTest.waitForContainerState(containerManager,
+        createContainerId(1),
+        org.apache.hadoop.yarn.api.records.ContainerState.RUNNING);
+
+    // Ensure all containers are running.
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    for (int i = 0; i < 2; i++) {
+      statList.add(createContainerId(i));
+    }
+    GetContainerStatusesRequest statRequest =
+        GetContainerStatusesRequest.newInstance(statList);
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    for (ContainerStatus status : containerStatuses) {
+      Assert.assertEquals(
+          org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+          status.getState());
+    }
+  }
+
+  /**
+   * Submit both a GUARANTEED and an OPPORTUNISTIC container, each of which
+   * requires more resources than available at the node, and make sure they
+   * are both queued.
+   * @throws Exception
+   */
+  @Test
+  public void testQueueMultipleContainers() throws Exception {
+    shouldDeleteWait = true;
+    containerManager.start();
+
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(3072, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.GUARANTEED)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(3072, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+
+    Thread.sleep(5000);
+
+    // Ensure both containers are queued.
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    for (int i = 0; i < 2; i++) {
+      statList.add(createContainerId(i));
+    }
+    GetContainerStatusesRequest statRequest =
+        GetContainerStatusesRequest.newInstance(statList);
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    for (ContainerStatus status : containerStatuses) {
+      Assert.assertEquals(
+          org.apache.hadoop.yarn.api.records.ContainerState.QUEUED,
+          status.getState());
+    }
+
+    // Ensure both containers are properly queued.
+    Assert.assertEquals(2, containerManager.getContext().getQueuingContext()
+        .getQueuedContainers().size());
+    Assert.assertEquals(1, ((QueuingContainerManagerImpl) containerManager)
+        .getNumQueuedGuaranteedContainers());
+    Assert.assertEquals(1, ((QueuingContainerManagerImpl) containerManager)
+        .getNumQueuedOpportunisticContainers());
+  }
+
+  /**
+   * Starts one OPPORTUNISTIC container that takes up the whole node's
+   * resources, and submit two more that will be queued.
+   * @throws Exception
+   */
+  @Test
+  public void testStartAndQueueMultipleContainers() throws Exception {
+    shouldDeleteWait = true;
+    containerManager.start();
+
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(2048, 1),
+            context.getContainerTokenSecretManager(), null,
             ExecutionType.OPPORTUNISTIC)));
     list.add(StartContainerRequest.newInstance(
         containerLaunchContext,
@@ -256,23 +316,95 @@ public class TestQueuingContainerManager extends TestContainerManager {
             user, BuilderUtils.newResource(1024, 1),
             context.getContainerTokenSecretManager(), null,
             ExecutionType.OPPORTUNISTIC)));
-    // GUARANTEED
+
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+
+    Thread.sleep(5000);
+
+    // Ensure first container is running and others are queued.
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    for (int i = 0; i < 3; i++) {
+      statList.add(createContainerId(i));
+    }
+    GetContainerStatusesRequest statRequest = GetContainerStatusesRequest
+        .newInstance(Arrays.asList(createContainerId(0)));
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    for (ContainerStatus status : containerStatuses) {
+      if (status.getContainerId().equals(createContainerId(0))) {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+            status.getState());
+      } else {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.QUEUED,
+            status.getState());
+      }
+    }
+
+    // Ensure two containers are properly queued.
+    Assert.assertEquals(2, containerManager.getContext().getQueuingContext()
+        .getQueuedContainers().size());
+    Assert.assertEquals(0, ((QueuingContainerManagerImpl) containerManager)
+        .getNumQueuedGuaranteedContainers());
+    Assert.assertEquals(2, ((QueuingContainerManagerImpl) containerManager)
+        .getNumQueuedOpportunisticContainers());
+  }
+
+  /**
+   * Submit two OPPORTUNISTIC and one GUARANTEED containers. The resources
+   * requests by each container as such that only one can run in parallel.
+   * Thus, the OPPORTUNISTIC container that started running, will be
+   * killed for the GUARANTEED container to start.
+   * Once the GUARANTEED container finishes its execution, the remaining
+   * OPPORTUNISTIC container will be executed.
+   * @throws Exception
+   */
+  @Test
+  public void testKillOpportunisticForGuaranteedContainer() throws Exception {
+    shouldDeleteWait = true;
+    containerManager.start();
+
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(2048, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
     list.add(StartContainerRequest.newInstance(
         containerLaunchContext,
-        createContainerToken(createContainerId(3), DUMMY_RM_IDENTIFIER,
+        createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
             context.getNodeId(),
-            user, BuilderUtils.newResource(1024, 1),
+            user, BuilderUtils.newResource(2048, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(2), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(2048, 1),
             context.getContainerTokenSecretManager(), null,
             ExecutionType.GUARANTEED)));
+
     StartContainersRequest allRequests =
         StartContainersRequest.newInstance(list);
-
     containerManager.startContainers(allRequests);
 
-    Thread.sleep(10000);
+    BaseContainerManagerTest.waitForNMContainerState(containerManager,
+        createContainerId(0), ContainerState.DONE, 30);
+    Thread.sleep(5000);
 
+    // Get container statuses. Container 0 should be killed, container 1
+    // should be queued and container 2 should be running.
     List<ContainerId> statList = new ArrayList<ContainerId>();
-    for (int i = 0; i < 4; i++) {
+    for (int i = 0; i < 3; i++) {
       statList.add(createContainerId(i));
     }
     GetContainerStatusesRequest statRequest =
@@ -280,12 +412,110 @@ public class TestQueuingContainerManager extends TestContainerManager {
     List<ContainerStatus> containerStatuses = containerManager
         .getContainerStatuses(statRequest).getContainerStatuses();
     for (ContainerStatus status : containerStatuses) {
-      // Ensure that the first opportunistic container is killed
       if (status.getContainerId().equals(createContainerId(0))) {
         Assert.assertTrue(status.getDiagnostics()
             .contains("Container killed by the ApplicationMaster"));
+      } else if (status.getContainerId().equals(createContainerId(1))) {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.QUEUED,
+            status.getState());
+      } else if (status.getContainerId().equals(createContainerId(2))) {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+            status.getState());
+      }
+      System.out.println("\nStatus : [" + status + "]\n");
+    }
+
+    // Make sure the remaining OPPORTUNISTIC container starts its execution.
+    BaseContainerManagerTest.waitForNMContainerState(containerManager,
+        createContainerId(2), ContainerState.DONE, 30);
+    Thread.sleep(5000);
+    statRequest = GetContainerStatusesRequest.newInstance(Arrays.asList(
+        createContainerId(1)));
+    ContainerStatus contStatus1 = containerManager.getContainerStatuses(
+        statRequest).getContainerStatuses().get(0);
+    Assert.assertEquals(
+        org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+        contStatus1.getState());
+  }
+
+  /**
+   * Submit three OPPORTUNISTIC containers that can run concurrently, and one
+   * GUARANTEED that needs to kill two of the OPPORTUNISTIC for it to run.
+   * @throws Exception
+   */
+  @Test
+  public void testKillMultipleOpportunisticContainers() throws Exception {
+    shouldDeleteWait = true;
+    containerManager.start();
+
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(512, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(512, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(2), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(512, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(3), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(1500, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.GUARANTEED)));
+
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+
+    BaseContainerManagerTest.waitForNMContainerState(containerManager,
+        createContainerId(0), ContainerState.DONE, 30);
+    Thread.sleep(5000);
+
+    // Get container statuses. Container 0 should be killed, container 1
+    // should be queued and container 2 should be running.
+    int killedContainers = 0;
+    int runningContainers = 0;
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    for (int i = 0; i < 4; i++) {
+      statList.add(createContainerId(i));
+    }
+    GetContainerStatusesRequest statRequest =
+        GetContainerStatusesRequest.newInstance(statList);
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    for (ContainerStatus status : containerStatuses) {
+      if (status.getDiagnostics().contains(
+          "Container killed by the ApplicationMaster")) {
+        killedContainers++;
+      }
+      if (status.getState() ==
+          org.apache.hadoop.yarn.api.records.ContainerState.RUNNING) {
+        runningContainers++;
       }
       System.out.println("\nStatus : [" + status + "]\n");
     }
+
+    Assert.assertEquals(2, killedContainers);
+    Assert.assertEquals(2, runningContainers);
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[25/51] [abbrv] hadoop git commit: YARN-5208. Run TestAMRMClient TestNMClient TestYarnClient TestClientRMTokens TestAMAuthorization tests with hadoop.security.token.service.use_ip enabled. (Rohith Sharma K S via wangda)

Posted by vv...@apache.org.
YARN-5208. Run TestAMRMClient TestNMClient TestYarnClient TestClientRMTokens TestAMAuthorization tests with hadoop.security.token.service.use_ip enabled. (Rohith Sharma K S via wangda)


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

Branch: refs/heads/YARN-3926
Commit: 244506f9c8953029283aa8e0eb2006ae2f30025e
Parents: d44f474
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Jun 10 09:34:32 2016 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Jun 10 09:34:32 2016 -0700

----------------------------------------------------------------------
 .../api/impl/TestAMRMClientOnRMRestart.java     | 16 +++++++++++--
 .../src/test/resources/core-site.xml            | 25 --------------------
 .../resourcemanager/TestAMAuthorization.java    | 22 +++++++++++++++++
 .../resourcemanager/TestClientRMTokens.java     | 22 +++++++++++++++++
 4 files changed, 58 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/244506f9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
index 0890396..719d9a1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
@@ -27,6 +27,7 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.security.SecurityUtil;
@@ -70,18 +71,22 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
 import org.apache.hadoop.yarn.util.Records;
+import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestAMRMClientOnRMRestart {
-  static Configuration conf = null;
+  static Configuration conf = new Configuration();
   static final int rolling_interval_sec = 13;
   static final long am_expire_ms = 4000;
 
   @BeforeClass
   public static void setup() throws Exception {
-    conf = new Configuration();
+    conf.setBoolean(
+        CommonConfigurationKeys.HADOOP_SECURITY_TOKEN_SERVICE_USE_IP, false);
+    SecurityUtil.setConfiguration(conf);
+
     conf.set(YarnConfiguration.RECOVERY_ENABLED, "true");
     conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
@@ -90,6 +95,13 @@ public class TestAMRMClientOnRMRestart {
     conf.setLong(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS, 0);
   }
 
+  @AfterClass
+  public static void tearDown() {
+    conf.setBoolean(
+        CommonConfigurationKeys.HADOOP_SECURITY_TOKEN_SERVICE_USE_IP, true);
+    SecurityUtil.setConfiguration(conf);
+  }
+
   // Test does major 6 steps verification.
   // Step-1 : AMRMClient send allocate request for 3 container requests
   // Step-2 : 3 containers are allocated by RM.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/244506f9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/resources/core-site.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/resources/core-site.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/resources/core-site.xml
deleted file mode 100644
index f0d3085..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/resources/core-site.xml
+++ /dev/null
@@ -1,25 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-
-<!-- Put site-specific property overrides in this file. -->
-
-<configuration>
-  <property>
-    <name>hadoop.security.token.service.use_ip</name>
-    <value>false</value>
-  </property>
-
-</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/244506f9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.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/TestAMAuthorization.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
index c51cd87..0f88c79 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
@@ -30,6 +30,7 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.io.DataInputByteBuffer;
 import org.apache.hadoop.security.AccessControlException;
@@ -63,7 +64,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -77,6 +80,25 @@ public class TestAMAuthorization {
   private final Configuration conf;
   private MockRM rm;
 
+  // Note : Any test case in ResourceManager package that creates a proxy has
+  // to be run with enabling hadoop.security.token.service.use_ip. And reset
+  // to false at the end of test class. See YARN-5208
+  @BeforeClass
+  public static void setUp() {
+    Configuration conf = new Configuration();
+    conf.setBoolean(
+        CommonConfigurationKeys.HADOOP_SECURITY_TOKEN_SERVICE_USE_IP, true);
+    SecurityUtil.setConfiguration(conf);
+  }
+
+  @AfterClass
+  public static void resetConf() {
+    Configuration conf = new Configuration();
+    conf.setBoolean(
+        CommonConfigurationKeys.HADOOP_SECURITY_TOKEN_SERVICE_USE_IP, false);
+    SecurityUtil.setConfiguration(conf);
+  }
+
   @Parameters
   public static Collection<Object[]> configs() {
     Configuration conf = new Configuration();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/244506f9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.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/TestClientRMTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
index c21db4e..65145a4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
@@ -37,11 +37,13 @@ import java.security.PrivilegedAction;
 import java.security.PrivilegedExceptionAction;
 
 import org.apache.hadoop.net.NetUtils;
+import org.junit.AfterClass;
 import org.junit.Assert;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.Text;
@@ -72,6 +74,7 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 
@@ -79,6 +82,25 @@ public class TestClientRMTokens {
 
   private static final Log LOG = LogFactory.getLog(TestClientRMTokens.class);
   
+  // Note : Any test case in ResourceManager package that creates a proxy has
+  // to be run with enabling hadoop.security.token.service.use_ip. And reset
+  // to false at the end of test class. See YARN-5208
+  @BeforeClass
+  public static void setUp() {
+    Configuration conf = new Configuration();
+    conf.setBoolean(
+        CommonConfigurationKeys.HADOOP_SECURITY_TOKEN_SERVICE_USE_IP, true);
+    SecurityUtil.setConfiguration(conf);
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    Configuration conf = new Configuration();
+    conf.setBoolean(
+        CommonConfigurationKeys.HADOOP_SECURITY_TOKEN_SERVICE_USE_IP, false);
+    SecurityUtil.setConfiguration(conf);
+  }
+
   @Before
   public void resetSecretManager() {
     RMDelegationTokenIdentifier.Renewer.setSecretManager(null, null);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[37/51] [abbrv] hadoop git commit: HADOOP-13213. Small Documentation bug with AuthenticatedURL in hadoop-auth. Contributed by Tom Ellis.

Posted by vv...@apache.org.
HADOOP-13213. Small Documentation bug with AuthenticatedURL in hadoop-auth. Contributed by Tom Ellis.

This closes #97.


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

Branch: refs/heads/YARN-3926
Commit: 8a1dcceccea5cfe9fb0c94daa2517cd64a16b3c0
Parents: 0bbb4dd
Author: Akira Ajisaka <aa...@apache.org>
Authored: Sat Jun 11 03:31:06 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Sat Jun 11 03:32:21 2016 +0900

----------------------------------------------------------------------
 .../hadoop/security/authentication/client/AuthenticatedURL.java  | 4 ++--
 hadoop-common-project/hadoop-auth/src/site/markdown/Examples.md  | 4 ++--
 2 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a1dccec/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java
index f87d9d8..6604c3f 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java
@@ -44,14 +44,14 @@ import java.util.Map;
  * URL url = new URL("http://foo:8080/bar");
  * AuthenticatedURL.Token token = new AuthenticatedURL.Token();
  * AuthenticatedURL aUrl = new AuthenticatedURL();
- * HttpURLConnection conn = new AuthenticatedURL(url, token).openConnection();
+ * HttpURLConnection conn = new AuthenticatedURL().openConnection(url, token);
  * ....
  * // use the 'conn' instance
  * ....
  *
  * // establishing a follow up connection using a token from the previous connection
  *
- * HttpURLConnection conn = new AuthenticatedURL(url, token).openConnection();
+ * HttpURLConnection conn = new AuthenticatedURL().openConnection(url, token);
  * ....
  * // use the 'conn' instance
  * ....

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a1dccec/hadoop-common-project/hadoop-auth/src/site/markdown/Examples.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/site/markdown/Examples.md b/hadoop-common-project/hadoop-auth/src/site/markdown/Examples.md
index 7efb642..4dad79d 100644
--- a/hadoop-common-project/hadoop-auth/src/site/markdown/Examples.md
+++ b/hadoop-common-project/hadoop-auth/src/site/markdown/Examples.md
@@ -57,9 +57,9 @@ Use the `AuthenticatedURL` class to obtain an authenticated HTTP connection:
     URL url = new URL("http://localhost:8080/hadoop-auth/kerberos/who");
     AuthenticatedURL.Token token = new AuthenticatedURL.Token();
     ...
-    HttpURLConnection conn = new AuthenticatedURL(url, token).openConnection();
+    HttpURLConnection conn = new AuthenticatedURL().openConnection(url, token);
     ...
-    conn = new AuthenticatedURL(url, token).openConnection();
+    conn = new AuthenticatedURL().openConnection(url, token);
     ...
 
 Building and Running the Examples


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[48/51] [abbrv] hadoop git commit: YARN-4172. Extend DominantResourceCalculator to account for all resources. (Varun Vasudev via wangda)

Posted by vv...@apache.org.
YARN-4172. Extend DominantResourceCalculator to account for all resources. (Varun Vasudev via wangda)


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

Branch: refs/heads/YARN-3926
Commit: f203d5d5195bf0e69dbb8a8875a43926c4d0af51
Parents: 9e518ef
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Jan 29 10:53:31 2016 +0800
Committer: Varun Vasudev <vv...@apache.org>
Committed: Mon Jun 13 14:33:13 2016 +0530

----------------------------------------------------------------------
 .../resource/DominantResourceCalculator.java    | 373 ++++++++++++++-----
 1 file changed, 272 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f203d5d5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
index 8f97fab..1f3a8c2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
@@ -20,57 +20,100 @@ package org.apache.hadoop.yarn.util.resource;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.UnitsConversionUtil;
+
+import java.util.HashSet;
+import java.util.Set;
 
 /**
- * A {@link ResourceCalculator} which uses the concept of  
+ * A {@link ResourceCalculator} which uses the concept of
  * <em>dominant resource</em> to compare multi-dimensional resources.
  *
- * Essentially the idea is that the in a multi-resource environment, 
- * the resource allocation should be determined by the dominant share 
- * of an entity (user or queue), which is the maximum share that the 
- * entity has been allocated of any resource. 
- * 
- * In a nutshell, it seeks to maximize the minimum dominant share across 
- * all entities. 
- * 
+ * Essentially the idea is that the in a multi-resource environment,
+ * the resource allocation should be determined by the dominant share
+ * of an entity (user or queue), which is the maximum share that the
+ * entity has been allocated of any resource.
+ *
+ * In a nutshell, it seeks to maximize the minimum dominant share across
+ * all entities.
+ *
  * For example, if user A runs CPU-heavy tasks and user B runs
- * memory-heavy tasks, it attempts to equalize CPU share of user A 
- * with Memory-share of user B. 
- * 
+ * memory-heavy tasks, it attempts to equalize CPU share of user A
+ * with Memory-share of user B.
+ *
  * In the single resource case, it reduces to max-min fairness for that resource.
- * 
+ *
  * See the Dominant Resource Fairness paper for more details:
  * www.cs.berkeley.edu/~matei/papers/2011/nsdi_drf.pdf
  */
 @Private
 @Unstable
 public class DominantResourceCalculator extends ResourceCalculator {
-  
+
+  private Set<String> resourceNames;
+
+  public DominantResourceCalculator() {
+    resourceNames = new HashSet<>();
+    resourceNames.add(ResourceInformation.MEMORY.getName());
+    resourceNames.add(ResourceInformation.VCORES.getName());
+  }
+
+  /**
+   * Compare two resources - if the value for every resource type for the lhs
+   * is greater than that of the rhs, return 1. If the value for every resource
+   * type in the lhs is less than the rhs, return -1. Otherwise, return 0
+   *
+   * @param lhs resource to be compared
+   * @param rhs resource to be compared
+   * @return 0, 1, or -1
+   */
+  private int compare(Resource lhs, Resource rhs) {
+    boolean lhsGreater = false;
+    boolean rhsGreater = false;
+    int ret = 0;
+
+    for (String rName : resourceNames) {
+      try {
+        ResourceInformation lhsResourceInformation =
+            lhs.getResourceInformation(rName);
+        ResourceInformation rhsResourceInformation =
+            rhs.getResourceInformation(rName);
+        int diff = lhsResourceInformation.compareTo(rhsResourceInformation);
+        if (diff >= 1) {
+          lhsGreater = true;
+        } else if (diff <= -1) {
+          rhsGreater = true;
+        }
+      } catch (YarnException ye) {
+        throw new IllegalArgumentException(
+            "Error getting resource information for " + rName, ye);
+      }
+    }
+    if (lhsGreater && rhsGreater) {
+      ret = 0;
+    } else if (lhsGreater) {
+      ret = 1;
+    } else if (rhsGreater) {
+      ret = -1;
+    }
+    return ret;
+  }
+
   @Override
   public int compare(Resource clusterResource, Resource lhs, Resource rhs) {
-    
     if (lhs.equals(rhs)) {
       return 0;
     }
-    
+
     if (isInvalidDivisor(clusterResource)) {
-      if ((lhs.getMemorySize() < rhs.getMemorySize() && lhs.getVirtualCores() > rhs
-          .getVirtualCores())
-          || (lhs.getMemorySize() > rhs.getMemorySize() && lhs.getVirtualCores() < rhs
-              .getVirtualCores())) {
-        return 0;
-      } else if (lhs.getMemorySize() > rhs.getMemorySize()
-          || lhs.getVirtualCores() > rhs.getVirtualCores()) {
-        return 1;
-      } else if (lhs.getMemorySize() < rhs.getMemorySize()
-          || lhs.getVirtualCores() < rhs.getVirtualCores()) {
-        return -1;
-      }
+      return this.compare(lhs, rhs);
     }
 
     float l = getResourceAsValue(clusterResource, lhs, true);
     float r = getResourceAsValue(clusterResource, rhs, true);
-    
+
     if (l < r) {
       return -1;
     } else if (l > r) {
@@ -78,135 +121,263 @@ public class DominantResourceCalculator extends ResourceCalculator {
     } else {
       l = getResourceAsValue(clusterResource, lhs, false);
       r = getResourceAsValue(clusterResource, rhs, false);
+
       if (l < r) {
         return -1;
       } else if (l > r) {
         return 1;
       }
     }
-    
+
     return 0;
   }
 
   /**
    * Use 'dominant' for now since we only have 2 resources - gives us a slight
    * performance boost.
-   * 
+   * <p></p>
    * Once we add more resources, we'll need a more complicated (and slightly
    * less performant algorithm).
    */
-  protected float getResourceAsValue(
-      Resource clusterResource, Resource resource, boolean dominant) {
-    // Just use 'dominant' resource
-    return (dominant) ?
-        Math.max(
-            (float)resource.getMemorySize() / clusterResource.getMemorySize(),
-            (float)resource.getVirtualCores() / clusterResource.getVirtualCores()
-            ) 
-        :
-          Math.min(
-              (float)resource.getMemorySize() / clusterResource.getMemorySize(),
-              (float)resource.getVirtualCores() / clusterResource.getVirtualCores()
-              ); 
-  }
-  
+  protected float getResourceAsValue(Resource clusterResource,
+      Resource resource, boolean dominant) {
+
+    float min = Float.MAX_VALUE;
+    float max = 0.0f;
+    for (String rName : resourceNames) {
+      try {
+        ResourceInformation clusterResourceResourceInformation =
+            clusterResource.getResourceInformation(rName);
+        ResourceInformation resourceInformation =
+            resource.getResourceInformation(rName);
+        Long resourceValue = UnitsConversionUtil
+            .convert(resourceInformation.getUnits(),
+                clusterResourceResourceInformation.getUnits(),
+                resourceInformation.getValue());
+        float tmp =
+            (float) resourceValue / (float) clusterResourceResourceInformation
+                .getValue();
+        min = min < tmp ? min : tmp;
+        max = max > tmp ? max : tmp;
+      } catch (YarnException ye) {
+        throw new IllegalArgumentException(
+            "Error getting resource information for " + resource, ye);
+      }
+    }
+    return (dominant) ? max : min;
+  }
+
   @Override
   public long computeAvailableContainers(Resource available, Resource required) {
-    return Math.min(
-        available.getMemorySize() / required.getMemorySize(),
-        available.getVirtualCores() / required.getVirtualCores());
+    long min = Long.MAX_VALUE;
+    for (String resource : resourceNames) {
+      try {
+        ResourceInformation availableResource =
+            available.getResourceInformation(resource);
+        ResourceInformation requiredResource =
+            required.getResourceInformation(resource);
+        Long requiredResourceValue = UnitsConversionUtil
+            .convert(requiredResource.getUnits(), availableResource.getUnits(),
+                requiredResource.getValue());
+        Long tmp = availableResource.getValue() / requiredResourceValue;
+        min = min < tmp ? min : tmp;
+      } catch (YarnException ye) {
+        throw new IllegalArgumentException(
+            "Error getting resource information for " + resource, ye);
+      }
+
+    }
+    return min > Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) min;
   }
 
   @Override
-  public float divide(Resource clusterResource, 
+  public float divide(Resource clusterResource,
       Resource numerator, Resource denominator) {
-    return 
-        getResourceAsValue(clusterResource, numerator, true) / 
+    return
+        getResourceAsValue(clusterResource, numerator, true) /
         getResourceAsValue(clusterResource, denominator, true);
   }
-  
+
   @Override
   public boolean isInvalidDivisor(Resource r) {
-    if (r.getMemorySize() == 0.0f || r.getVirtualCores() == 0.0f) {
-      return true;
+    for (String resource : resourceNames) {
+      try {
+        if (r.getResourceValue(resource).equals(0L)) {
+          return true;
+        }
+      } catch (YarnException ye) {
+        throw new IllegalArgumentException(
+            "Error getting resource value for " + resource, ye);
+      }
     }
     return false;
   }
 
   @Override
   public float ratio(Resource a, Resource b) {
-    return Math.max(
-        (float)a.getMemorySize()/b.getMemorySize(),
-        (float)a.getVirtualCores()/b.getVirtualCores()
-        );
+    float ratio = 0.0f;
+    for (String resource : resourceNames) {
+      try {
+        ResourceInformation aResourceInformation =
+            a.getResourceInformation(resource);
+        ResourceInformation bResourceInformation =
+            b.getResourceInformation(resource);
+        Long bResourceValue = UnitsConversionUtil
+            .convert(bResourceInformation.getUnits(),
+                aResourceInformation.getUnits(),
+                bResourceInformation.getValue());
+        float tmp =
+            (float) aResourceInformation.getValue() / (float) bResourceValue;
+        ratio = ratio > tmp ? ratio : tmp;
+      } catch (YarnException ye) {
+        throw new IllegalArgumentException(
+            "Error getting resource information for " + resource, ye);
+      }
+    }
+    return ratio;
   }
 
   @Override
   public Resource divideAndCeil(Resource numerator, long denominator) {
-    return Resources.createResource(
-        divideAndCeil(numerator.getMemorySize(), denominator),
-        divideAndCeil(numerator.getVirtualCores(), denominator)
-        );
+    Resource ret = Resources.createResource(0, 0);
+    for (String resource : resourceNames) {
+      try {
+        ResourceInformation resourceInformation = ResourceInformation
+            .newInstance(numerator.getResourceInformation(resource));
+        resourceInformation.setValue(
+            divideAndCeil(resourceInformation.getValue(), denominator));
+        ret.setResourceInformation(resource, resourceInformation);
+      } catch (YarnException ye) {
+        throw new IllegalArgumentException(
+            "Error getting resource information for " + resource, ye);
+      }
+    }
+    return ret;
   }
 
   @Override
   public Resource normalize(Resource r, Resource minimumResource,
-                            Resource maximumResource, Resource stepFactor) {
-    long normalizedMemory = Math.min(
-      roundUp(
-        Math.max(r.getMemorySize(), minimumResource.getMemorySize()),
-        stepFactor.getMemorySize()),
-      maximumResource.getMemorySize());
-    long normalizedCores = Math.min(
-      roundUp(
-        Math.max(r.getVirtualCores(), minimumResource.getVirtualCores()),
-        stepFactor.getVirtualCores()),
-      maximumResource.getVirtualCores());
-    return Resources.createResource(normalizedMemory,
-      normalizedCores);
+      Resource maximumResource, Resource stepFactor) {
+    Resource ret = Resources.createResource(0, 0);
+    for (String resource : resourceNames) {
+      try {
+        ResourceInformation rResourceInformation =
+            r.getResourceInformation(resource);
+        ResourceInformation minimumResourceInformation =
+            minimumResource.getResourceInformation(resource);
+        ResourceInformation maximumResourceInformation =
+            maximumResource.getResourceInformation(resource);
+        ResourceInformation stepFactorResourceInformation =
+            stepFactor.getResourceInformation(resource);
+        ResourceInformation tmp =
+            ResourceInformation.newInstance(rResourceInformation);
+
+        Long rValue = rResourceInformation.getValue();
+        Long minimumValue = UnitsConversionUtil
+            .convert(minimumResourceInformation.getUnits(),
+                rResourceInformation.getUnits(),
+                minimumResourceInformation.getValue());
+        Long maximumValue = UnitsConversionUtil
+            .convert(maximumResourceInformation.getUnits(),
+                rResourceInformation.getUnits(),
+                maximumResourceInformation.getValue());
+        Long stepFactorValue = UnitsConversionUtil
+            .convert(stepFactorResourceInformation.getUnits(),
+                rResourceInformation.getUnits(),
+                stepFactorResourceInformation.getValue());
+
+        tmp.setValue(
+            Math.min(roundUp(Math.max(rValue, minimumValue), stepFactorValue),
+                maximumValue));
+        ret.setResourceInformation(resource, tmp);
+      } catch (YarnException ye) {
+        throw new IllegalArgumentException(
+            "Error getting resource information for " + resource, ye);
+      }
+    }
+    return ret;
   }
 
   @Override
   public Resource roundUp(Resource r, Resource stepFactor) {
-    return Resources.createResource(
-        roundUp(r.getMemorySize(), stepFactor.getMemorySize()),
-        roundUp(r.getVirtualCores(), stepFactor.getVirtualCores())
-        );
+    return this.rounding(r, stepFactor, true);
   }
 
   @Override
   public Resource roundDown(Resource r, Resource stepFactor) {
-    return Resources.createResource(
-        roundDown(r.getMemorySize(), stepFactor.getMemorySize()),
-        roundDown(r.getVirtualCores(), stepFactor.getVirtualCores())
-        );
+    return this.rounding(r, stepFactor, false);
+  }
+
+  private Resource rounding(Resource r, Resource stepFactor, boolean roundUp) {
+    Resource ret = Resources.createResource(0, 0);
+    for (String resource : resourceNames) {
+      try {
+        ResourceInformation rResourceInformation =
+            r.getResourceInformation(resource);
+        ResourceInformation stepFactorResourceInformation =
+            stepFactor.getResourceInformation(resource);
+        ResourceInformation tmp =
+            ResourceInformation.newInstance(rResourceInformation);
+
+        Long rValue = rResourceInformation.getValue();
+        Long stepFactorValue = UnitsConversionUtil
+            .convert(stepFactorResourceInformation.getUnits(),
+                rResourceInformation.getUnits(),
+                stepFactorResourceInformation.getValue());
+
+        Long value = roundUp ? roundUp(rValue, stepFactorValue) :
+            roundDown(rValue, stepFactorValue);
+        tmp.setValue(value);
+        ret.setResourceInformation(resource, tmp);
+      } catch (YarnException ye) {
+        throw new IllegalArgumentException(
+            "Error getting resource information for " + resource, ye);
+      }
+    }
+    return ret;
   }
 
   @Override
   public Resource multiplyAndNormalizeUp(Resource r, double by,
       Resource stepFactor) {
-    return Resources.createResource(
-        roundUp(
-            (int)Math.ceil(r.getMemorySize() * by), stepFactor.getMemorySize()),
-        roundUp(
-            (int)Math.ceil(r.getVirtualCores() * by), 
-            stepFactor.getVirtualCores())
-        );
+    return this.multiplyAndNormalize(r, by, stepFactor, true);
   }
 
   @Override
   public Resource multiplyAndNormalizeDown(Resource r, double by,
       Resource stepFactor) {
-    return Resources.createResource(
-        roundDown(
-            (int)(r.getMemorySize() * by),
-            stepFactor.getMemorySize()
-            ),
-        roundDown(
-            (int)(r.getVirtualCores() * by), 
-            stepFactor.getVirtualCores()
-            )
-        );
+    return this.multiplyAndNormalize(r, by, stepFactor, false);
+  }
+
+  private Resource multiplyAndNormalize(Resource r, double by,
+      Resource stepFactor, boolean roundUp) {
+    Resource ret = Resources.createResource(0, 0);
+    for (String resource : resourceNames) {
+      try {
+        ResourceInformation rResourceInformation =
+            r.getResourceInformation(resource);
+        ResourceInformation stepFactorResourceInformation =
+            stepFactor.getResourceInformation(resource);
+        ResourceInformation tmp =
+            ResourceInformation.newInstance(rResourceInformation);
+
+        Long rValue = rResourceInformation.getValue();
+        Long stepFactorValue = UnitsConversionUtil
+            .convert(stepFactorResourceInformation.getUnits(),
+                rResourceInformation.getUnits(),
+                stepFactorResourceInformation.getValue());
+
+        Long value =
+            roundUp ? roundUp((long) Math.ceil(rValue * by), stepFactorValue) :
+                roundDown((long) (rValue * by), stepFactorValue);
+        tmp.setValue(value);
+        ret.setResourceInformation(resource, tmp);
+      } catch (YarnException ye) {
+        throw new IllegalArgumentException(
+            "Error getting resource information for " + resource, ye);
+      }
+    }
+    return ret;
   }
 
   @Override


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[42/51] [abbrv] hadoop git commit: YARN-5212. Run existing ContainerManager tests using QueuingContainerManagerImpl. (Konstantinos Karanasos via asuresh)

Posted by vv...@apache.org.
YARN-5212. Run existing ContainerManager tests using QueuingContainerManagerImpl. (Konstantinos Karanasos via asuresh)


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

Branch: refs/heads/YARN-3926
Commit: 7dae2b3bc4cebbb186b3edd14e31074be02af329
Parents: 5143277
Author: Arun Suresh <as...@apache.org>
Authored: Sun Jun 12 10:05:37 2016 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Sun Jun 12 10:05:37 2016 -0700

----------------------------------------------------------------------
 .../TestContainerManagerRegression.java         | 84 ++++++++++++++++++++
 1 file changed, 84 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7dae2b3b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRegression.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRegression.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRegression.java
new file mode 100644
index 0000000..71af76f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRegression.java
@@ -0,0 +1,84 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.security.NMTokenIdentifier;
+import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.queuing.QueuingContainerManagerImpl;
+
+/**
+ * Test class that invokes all test cases of {@link TestContainerManager} while
+ * using the {@link QueuingContainerManagerImpl}. The goal is to assert that
+ * no regression is introduced in the existing cases when no queuing of tasks at
+ * the NMs is involved.
+ */
+public class TestContainerManagerRegression extends TestContainerManager {
+
+  public TestContainerManagerRegression()
+      throws UnsupportedFileSystemException {
+    super();
+  }
+
+  static {
+    LOG = LogFactory.getLog(TestContainerManagerRegression.class);
+  }
+
+  @Override
+  protected ContainerManagerImpl createContainerManager(
+      DeletionService delSrvc) {
+    return new QueuingContainerManagerImpl(context, exec, delSrvc,
+        nodeStatusUpdater, metrics, dirsHandler) {
+      @Override
+      public void
+          setBlockNewContainerRequests(boolean blockNewContainerRequests) {
+        // do nothing
+      }
+
+      @Override
+      protected UserGroupInformation getRemoteUgi() throws YarnException {
+        ApplicationId appId = ApplicationId.newInstance(0, 0);
+        ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(
+            appId, 1);
+        UserGroupInformation ugi = UserGroupInformation.createRemoteUser(
+            appAttemptId.toString());
+        ugi.addTokenIdentifier(new NMTokenIdentifier(appAttemptId, context
+            .getNodeId(), user, context.getNMTokenSecretManager()
+                .getCurrentKey().getKeyId()));
+        return ugi;
+      }
+
+      @Override
+      protected void authorizeGetAndStopContainerRequest(
+          ContainerId containerId, Container container, boolean stopRequest,
+          NMTokenIdentifier identifier) throws YarnException {
+        if (container == null || container.getUser().equals("Fail")) {
+          throw new YarnException("Reject this container");
+        }
+      }
+    };
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[11/51] [abbrv] hadoop git commit: MAPREDUCE-6240. Hadoop client displays confusing error message. (gera)

Posted by vv...@apache.org.
MAPREDUCE-6240. Hadoop client displays confusing error message. (gera)


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

Branch: refs/heads/YARN-3926
Commit: 0af96a1c08594c809ecb254cee4f60dd22399772
Parents: 1ee9ea0
Author: Gera Shegalov <ge...@apache.org>
Authored: Sat May 28 22:01:07 2016 -0700
Committer: Gera Shegalov <ge...@apache.org>
Committed: Wed Jun 8 12:59:37 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/mapreduce/Cluster.java    | 15 ++++++-----
 .../TestClientProtocolProviderImpls.java        | 26 +++++++++++++++++---
 2 files changed, 32 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0af96a1c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java
index 9563c0b..6ca918d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java
@@ -102,6 +102,10 @@ public class Cluster {
       throws IOException {
 
     initProviderList();
+    final IOException initEx = new IOException(
+        "Cannot initialize Cluster. Please check your configuration for "
+            + MRConfig.FRAMEWORK_NAME
+            + " and the correspond server addresses.");
     for (ClientProtocolProvider provider : providerList) {
       LOG.debug("Trying ClientProtocolProvider : "
           + provider.getClass().getName());
@@ -124,16 +128,15 @@ public class Cluster {
               + " as the ClientProtocolProvider - returned null protocol");
         }
       } catch (Exception e) {
-        LOG.info("Failed to use " + provider.getClass().getName()
-            + " due to error: ", e);
+        final String errMsg = "Failed to use " + provider.getClass().getName()
+            + " due to error: ";
+        initEx.addSuppressed(new IOException(errMsg, e));
+        LOG.info(errMsg, e);
       }
     }
 
     if (null == clientProtocolProvider || null == client) {
-      throw new IOException(
-          "Cannot initialize Cluster. Please check your configuration for "
-              + MRConfig.FRAMEWORK_NAME
-              + " and the correspond server addresses.");
+      throw initEx;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0af96a1c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestClientProtocolProviderImpls.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestClientProtocolProviderImpls.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestClientProtocolProviderImpls.java
index 6ad76e9..500e133 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestClientProtocolProviderImpls.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestClientProtocolProviderImpls.java
@@ -18,17 +18,20 @@
 
 package org.apache.hadoop.mapreduce;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.mapred.LocalJobRunner;
 import org.apache.hadoop.mapred.YARNRunner;
 import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+import org.apache.hadoop.util.StringUtils;
 import org.junit.Test;
 
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 public class TestClientProtocolProviderImpls {
 
   @Test
@@ -76,4 +79,21 @@ public class TestClientProtocolProviderImpls {
           "Cannot initialize Cluster. Please check"));
     }
   }
+
+  @Test
+  public void testClusterExceptionRootCause() throws Exception {
+    final Configuration conf = new Configuration();
+    conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_FRAMEWORK_NAME);
+    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "nosuchfs:///");
+    conf.set(JTConfig.JT_IPC_ADDRESS, "local");
+    try {
+      new Cluster(conf);
+      fail("Cluster init should fail because of non-existing FileSystem");
+    } catch (IOException ioEx) {
+      final String stackTrace = StringUtils.stringifyException(ioEx);
+      assertTrue("No root cause detected",
+          stackTrace.contains(UnsupportedFileSystemException.class.getName())
+              && stackTrace.contains("nosuchfs"));
+    }
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[41/51] [abbrv] hadoop git commit: YARN-5124. Modify AMRMClient to set the ExecutionType in the ResourceRequest. (asuresh)

Posted by vv...@apache.org.
YARN-5124. Modify AMRMClient to set the ExecutionType in the ResourceRequest. (asuresh)


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

Branch: refs/heads/YARN-3926
Commit: 51432779588fdd741b4840601f5db637ec783d92
Parents: 5279af7
Author: Arun Suresh <as...@apache.org>
Authored: Sun Jun 12 09:42:38 2016 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Sun Jun 12 09:42:38 2016 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/client/api/AMRMClient.java      |  43 +-
 .../yarn/client/api/async/AMRMClientAsync.java  |  17 +
 .../yarn/client/api/impl/AMRMClientImpl.java    | 294 +++------
 .../client/api/impl/RemoteRequestsTable.java    | 332 ++++++++++
 .../client/api/impl/BaseAMRMProxyE2ETest.java   | 197 ++++++
 .../yarn/client/api/impl/TestAMRMClient.java    |  26 +-
 .../impl/TestAMRMClientContainerRequest.java    |  54 +-
 .../yarn/client/api/impl/TestAMRMProxy.java     | 171 +----
 .../api/impl/TestDistributedScheduling.java     | 644 ++++++++++++-------
 .../yarn/client/api/impl/TestNMClient.java      |   7 +-
 .../records/impl/pb/ResourceRequestPBImpl.java  |   5 +-
 11 files changed, 1204 insertions(+), 586 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/51432779/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
index 3ec0899..5f362c8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -109,7 +110,7 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
     final Priority priority;
     final boolean relaxLocality;
     final String nodeLabelsExpression;
-    final ExecutionType executionType;
+    final ExecutionTypeRequest executionTypeRequest;
     
     /**
      * Instantiates a {@link ContainerRequest} with the given constraints and
@@ -180,7 +181,7 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
         Priority priority, boolean relaxLocality, String nodeLabelsExpression) {
       this(capability, nodes, racks, priority, relaxLocality,
           nodeLabelsExpression,
-          ExecutionType.GUARANTEED);
+          ExecutionTypeRequest.newInstance());
     }
           
     /**
@@ -203,12 +204,12 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
      * @param nodeLabelsExpression
      *          Set node labels to allocate resource, now we only support
      *          asking for only a single node label
-     * @param executionType
+     * @param executionTypeRequest
      *          Set the execution type of the container request.
      */
     public ContainerRequest(Resource capability, String[] nodes, String[] racks,
         Priority priority, boolean relaxLocality, String nodeLabelsExpression,
-        ExecutionType executionType) {
+        ExecutionTypeRequest executionTypeRequest) {
       // Validate request
       Preconditions.checkArgument(capability != null,
           "The Resource to be requested for each container " +
@@ -226,7 +227,7 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
       this.priority = priority;
       this.relaxLocality = relaxLocality;
       this.nodeLabelsExpression = nodeLabelsExpression;
-      this.executionType = executionType;
+      this.executionTypeRequest = executionTypeRequest;
     }
     
     public Resource getCapability() {
@@ -253,15 +254,16 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
       return nodeLabelsExpression;
     }
     
-    public ExecutionType getExecutionType() {
-      return executionType;
+    public ExecutionTypeRequest getExecutionTypeRequest() {
+      return executionTypeRequest;
     }
 
     public String toString() {
       StringBuilder sb = new StringBuilder();
       sb.append("Capability[").append(capability).append("]");
       sb.append("Priority[").append(priority).append("]");
-      sb.append("ExecutionType[").append(executionType).append("]");
+      sb.append("ExecutionTypeRequest[").append(executionTypeRequest)
+          .append("]");
       return sb.toString();
     }
   }
@@ -388,10 +390,35 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
    * collection, requests will be returned in the same order as they were added.
    * @return Collection of request matching the parameters
    */
+  @InterfaceStability.Evolving
   public abstract List<? extends Collection<T>> getMatchingRequests(
                                            Priority priority, 
                                            String resourceName, 
                                            Resource capability);
+
+  /**
+   * Get outstanding <code>ContainerRequest</code>s matching the given
+   * parameters. These ContainerRequests should have been added via
+   * <code>addContainerRequest</code> earlier in the lifecycle. For performance,
+   * the AMRMClient may return its internal collection directly without creating
+   * a copy. Users should not perform mutable operations on the return value.
+   * Each collection in the list contains requests with identical
+   * <code>Resource</code> size that fit in the given capability. In a
+   * collection, requests will be returned in the same order as they were added.
+   * specify an <code>ExecutionType</code> .
+   * @param priority Priority
+   * @param resourceName Location
+   * @param executionType ExecutionType
+   * @param capability Capability
+   * @return Collection of request matching the parameters
+   */
+  @InterfaceStability.Evolving
+  public List<? extends Collection<T>> getMatchingRequests(
+      Priority priority, String resourceName, ExecutionType executionType,
+      Resource capability) {
+    throw new UnsupportedOperationException("The sub-class extending" +
+        " AMRMClient is expected to implement this !!");
+  }
   
   /**
    * Update application's blacklist with addition or removal resources.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51432779/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
index 3c8f923..2f95156 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -196,6 +197,22 @@ extends AbstractService {
                                                    Priority priority, 
                                                    String resourceName, 
                                                    Resource capability);
+
+  /**
+   * Returns all matching ContainerRequests that match the given Priority,
+   * ResourceName, ExecutionType and Capability.
+   * @param priority Priority.
+   * @param resourceName Location.
+   * @param executionType ExecutionType.
+   * @param capability Capability.
+   * @return All matching ContainerRequests
+   */
+  public List<? extends Collection<T>> getMatchingRequests(
+      Priority priority, String resourceName, ExecutionType executionType,
+      Resource capability) {
+    return client.getMatchingRequests(priority, resourceName,
+        executionType, capability);
+  }
   
   /**
    * Registers this application master with the resource manager. On successful

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51432779/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
index 4366c25..4145944 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
@@ -19,19 +19,19 @@
 package org.apache.hadoop.yarn.client.api.impl;
 
 import java.io.IOException;
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.LinkedHashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.AbstractMap.SimpleEntry;
 
@@ -54,6 +54,8 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -102,7 +104,7 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
   protected final Set<String> blacklistAdditions = new HashSet<String>();
   protected final Set<String> blacklistRemovals = new HashSet<String>();
   
-  class ResourceRequestInfo {
+  static class ResourceRequestInfo<T> {
     ResourceRequest remoteRequest;
     LinkedHashSet<T> containerRequests;
     
@@ -115,11 +117,12 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
     }
   }
 
-
   /**
    * Class compares Resource by memory then cpu in reverse order
    */
-  class ResourceReverseMemoryThenCpuComparator implements Comparator<Resource> {
+  static class ResourceReverseMemoryThenCpuComparator implements
+      Comparator<Resource>, Serializable {
+    static final long serialVersionUID = 12345L;
     @Override
     public int compare(Resource arg0, Resource arg1) {
       long mem0 = arg0.getMemorySize();
@@ -141,7 +144,7 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
       return -1;
     }    
   }
-  
+
   static boolean canFit(Resource arg0, Resource arg1) {
     long mem0 = arg0.getMemorySize();
     long mem1 = arg1.getMemorySize();
@@ -150,17 +153,8 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
     
     return (mem0 <= mem1 && cpu0 <= cpu1);
   }
-  
-  //Key -> Priority
-  //Value -> Map
-  //Key->ResourceName (e.g., nodename, rackname, *)
-  //Value->Map
-  //Key->Resource Capability
-  //Value->ResourceRequest
-  protected final 
-  Map<Priority, Map<String, TreeMap<Resource, ResourceRequestInfo>>>
-    remoteRequestsTable =
-    new TreeMap<Priority, Map<String, TreeMap<Resource, ResourceRequestInfo>>>();
+
+  final RemoteRequestsTable remoteRequestsTable = new RemoteRequestsTable<T>();
 
   protected final Set<ResourceRequest> ask = new TreeSet<ResourceRequest>(
       new org.apache.hadoop.yarn.api.records.ResourceRequest.ResourceRequestComparator());
@@ -185,6 +179,12 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
     super(AMRMClientImpl.class.getName());
   }
 
+  @VisibleForTesting
+  AMRMClientImpl(ApplicationMasterProtocol protocol) {
+    super(AMRMClientImpl.class.getName());
+    this.rmClient = protocol;
+  }
+
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
     RackResolver.init(conf);
@@ -195,8 +195,10 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
   protected void serviceStart() throws Exception {
     final YarnConfiguration conf = new YarnConfiguration(getConfig());
     try {
-      rmClient =
-          ClientRMProxy.createRMProxy(conf, ApplicationMasterProtocol.class);
+      if (rmClient == null) {
+        rmClient = ClientRMProxy.createRMProxy(
+            conf, ApplicationMasterProtocol.class);
+      }
     } catch (IOException e) {
       throw new YarnRuntimeException(e);
     }
@@ -263,7 +265,8 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
           // RPC layer is using it to send info across
           askList.add(ResourceRequest.newInstance(r.getPriority(),
               r.getResourceName(), r.getCapability(), r.getNumContainers(),
-              r.getRelaxLocality(), r.getNodeLabelExpression()));
+              r.getRelaxLocality(), r.getNodeLabelExpression(),
+              r.getExecutionTypeRequest()));
         }
         List<ContainerResourceChangeRequest> increaseList = new ArrayList<>();
         List<ContainerResourceChangeRequest> decreaseList = new ArrayList<>();
@@ -315,13 +318,11 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
         synchronized (this) {
           release.addAll(this.pendingRelease);
           blacklistAdditions.addAll(this.blacklistedNodes);
-          for (Map<String, TreeMap<Resource, ResourceRequestInfo>> rr : remoteRequestsTable
-            .values()) {
-            for (Map<Resource, ResourceRequestInfo> capabalities : rr.values()) {
-              for (ResourceRequestInfo request : capabalities.values()) {
-                addResourceRequestToAsk(request.remoteRequest);
-              }
-            }
+          @SuppressWarnings("unchecked")
+          Iterator<ResourceRequestInfo<T>> reqIter =
+              remoteRequestsTable.iterator();
+          while (reqIter.hasNext()) {
+            addResourceRequestToAsk(reqIter.next().remoteRequest);
           }
           change.putAll(this.pendingChange);
         }
@@ -517,26 +518,28 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
             + joiner.join(req.getNodes()));        
       }
       for (String node : dedupedNodes) {
-        addResourceRequest(req.getPriority(), node, req.getCapability(), req,
-            true, req.getNodeLabelExpression());
+        addResourceRequest(req.getPriority(), node,
+            req.getExecutionTypeRequest(), req.getCapability(), req, true,
+            req.getNodeLabelExpression());
       }
     }
 
     for (String rack : dedupedRacks) {
-      addResourceRequest(req.getPriority(), rack, req.getCapability(), req,
-          true, req.getNodeLabelExpression());
+      addResourceRequest(req.getPriority(), rack, req.getExecutionTypeRequest(),
+          req.getCapability(), req, true, req.getNodeLabelExpression());
     }
 
     // Ensure node requests are accompanied by requests for
     // corresponding rack
     for (String rack : inferredRacks) {
-      addResourceRequest(req.getPriority(), rack, req.getCapability(), req,
-          req.getRelaxLocality(), req.getNodeLabelExpression());
+      addResourceRequest(req.getPriority(), rack, req.getExecutionTypeRequest(),
+          req.getCapability(), req, req.getRelaxLocality(),
+          req.getNodeLabelExpression());
     }
-
     // Off-switch
-    addResourceRequest(req.getPriority(), ResourceRequest.ANY, 
-        req.getCapability(), req, req.getRelaxLocality(), req.getNodeLabelExpression());
+    addResourceRequest(req.getPriority(), ResourceRequest.ANY,
+        req.getExecutionTypeRequest(), req.getCapability(), req,
+        req.getRelaxLocality(), req.getNodeLabelExpression());
   }
 
   @Override
@@ -552,16 +555,18 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
     // Update resource requests
     if (req.getNodes() != null) {
       for (String node : new HashSet<String>(req.getNodes())) {
-        decResourceRequest(req.getPriority(), node, req.getCapability(), req);
+        decResourceRequest(req.getPriority(), node,
+            req.getExecutionTypeRequest(), req.getCapability(), req);
       }
     }
 
     for (String rack : allRacks) {
-      decResourceRequest(req.getPriority(), rack, req.getCapability(), req);
+      decResourceRequest(req.getPriority(), rack,
+          req.getExecutionTypeRequest(), req.getCapability(), req);
     }
 
     decResourceRequest(req.getPriority(), ResourceRequest.ANY,
-        req.getCapability(), req);
+        req.getExecutionTypeRequest(), req.getCapability(), req);
   }
 
   @Override
@@ -601,47 +606,38 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
   public synchronized int getClusterNodeCount() {
     return clusterNodeCount;
   }
-  
+
+  @Override
+  public synchronized List<? extends Collection<T>> getMatchingRequests(
+      Priority priority,
+      String resourceName,
+      Resource capability) {
+    return getMatchingRequests(priority, resourceName,
+        ExecutionType.GUARANTEED, capability);
+  }
+
   @Override
   public synchronized List<? extends Collection<T>> getMatchingRequests(
-                                          Priority priority, 
-                                          String resourceName, 
-                                          Resource capability) {
+      Priority priority, String resourceName, ExecutionType executionType,
+      Resource capability) {
     Preconditions.checkArgument(capability != null,
         "The Resource to be requested should not be null ");
     Preconditions.checkArgument(priority != null,
         "The priority at which to request containers should not be null ");
     List<LinkedHashSet<T>> list = new LinkedList<LinkedHashSet<T>>();
-    Map<String, TreeMap<Resource, ResourceRequestInfo>> remoteRequests = 
-        this.remoteRequestsTable.get(priority);
-    if (remoteRequests == null) {
-      return list;
-    }
-    TreeMap<Resource, ResourceRequestInfo> reqMap = remoteRequests
-        .get(resourceName);
-    if (reqMap == null) {
-      return list;
-    }
 
-    ResourceRequestInfo resourceRequestInfo = reqMap.get(capability);
-    if (resourceRequestInfo != null &&
-        !resourceRequestInfo.containerRequests.isEmpty()) {
-      list.add(resourceRequestInfo.containerRequests);
-      return list;
-    }
-    
-    // no exact match. Container may be larger than what was requested.
-    // get all resources <= capability. map is reverse sorted. 
-    SortedMap<Resource, ResourceRequestInfo> tailMap = 
-                                                  reqMap.tailMap(capability);
-    for(Map.Entry<Resource, ResourceRequestInfo> entry : tailMap.entrySet()) {
-      if (canFit(entry.getKey(), capability) &&
-          !entry.getValue().containerRequests.isEmpty()) {
-        // match found that fits in the larger resource
-        list.add(entry.getValue().containerRequests);
+    @SuppressWarnings("unchecked")
+    List<ResourceRequestInfo<T>> matchingRequests =
+        this.remoteRequestsTable.getMatchingRequests(priority, resourceName,
+            executionType, capability);
+    // If no exact match. Container may be larger than what was requested.
+    // get all resources <= capability. map is reverse sorted.
+    for (ResourceRequestInfo<T> resReqInfo : matchingRequests) {
+      if (canFit(resReqInfo.remoteRequest.getCapability(), capability) &&
+        !resReqInfo.containerRequests.isEmpty()) {
+        list.add(resReqInfo.containerRequests);
       }
     }
-    
     // no match found
     return list;          
   }
@@ -663,34 +659,30 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
     
     return racks;
   }
-  
+
   /**
    * ContainerRequests with locality relaxation cannot be made at the same
    * priority as ContainerRequests without locality relaxation.
    */
   private void checkLocalityRelaxationConflict(Priority priority,
       Collection<String> locations, boolean relaxLocality) {
-    Map<String, TreeMap<Resource, ResourceRequestInfo>> remoteRequests =
-        this.remoteRequestsTable.get(priority);
-    if (remoteRequests == null) {
-      return;
-    }
     // Locality relaxation will be set to relaxLocality for all implicitly
     // requested racks. Make sure that existing rack requests match this.
-    for (String location : locations) {
-        TreeMap<Resource, ResourceRequestInfo> reqs =
-            remoteRequests.get(location);
-        if (reqs != null && !reqs.isEmpty()) {
-          boolean existingRelaxLocality =
-              reqs.values().iterator().next().remoteRequest.getRelaxLocality();
-          if (relaxLocality != existingRelaxLocality) {
-            throw new InvalidContainerRequestException("Cannot submit a "
-                + "ContainerRequest asking for location " + location
-                + " with locality relaxation " + relaxLocality + " when it has "
-                + "already been requested with locality relaxation " + existingRelaxLocality);
-          }
-        }
+
+    @SuppressWarnings("unchecked")
+    List<ResourceRequestInfo> allCapabilityMaps =
+        remoteRequestsTable.getAllResourceRequestInfos(priority, locations);
+    for (ResourceRequestInfo reqs : allCapabilityMaps) {
+      ResourceRequest remoteRequest = reqs.remoteRequest;
+      boolean existingRelaxLocality = remoteRequest.getRelaxLocality();
+      if (relaxLocality != existingRelaxLocality) {
+        throw new InvalidContainerRequestException("Cannot submit a "
+            + "ContainerRequest asking for location "
+            + remoteRequest.getResourceName() + " with locality relaxation "
+            + relaxLocality + " when it has already been requested"
+            + "with locality relaxation " + existingRelaxLocality);
       }
+    }
   }
   
   /**
@@ -747,46 +739,13 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
     ask.add(remoteRequest);
   }
 
-  private void
-      addResourceRequest(Priority priority, String resourceName,
-          Resource capability, T req, boolean relaxLocality,
-          String labelExpression) {
-    Map<String, TreeMap<Resource, ResourceRequestInfo>> remoteRequests =
-      this.remoteRequestsTable.get(priority);
-    if (remoteRequests == null) {
-      remoteRequests = 
-          new HashMap<String, TreeMap<Resource, ResourceRequestInfo>>();
-      this.remoteRequestsTable.put(priority, remoteRequests);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Added priority=" + priority);
-      }
-    }
-    TreeMap<Resource, ResourceRequestInfo> reqMap = 
-                                          remoteRequests.get(resourceName);
-    if (reqMap == null) {
-      // capabilities are stored in reverse sorted order. smallest last.
-      reqMap = new TreeMap<Resource, ResourceRequestInfo>(
-          new ResourceReverseMemoryThenCpuComparator());
-      remoteRequests.put(resourceName, reqMap);
-    }
-    ResourceRequestInfo resourceRequestInfo = reqMap.get(capability);
-    if (resourceRequestInfo == null) {
-      resourceRequestInfo =
-          new ResourceRequestInfo(priority, resourceName, capability,
-              relaxLocality);
-      reqMap.put(capability, resourceRequestInfo);
-    }
-    
-    resourceRequestInfo.remoteRequest.setNumContainers(
-         resourceRequestInfo.remoteRequest.getNumContainers() + 1);
-
-    if (relaxLocality) {
-      resourceRequestInfo.containerRequests.add(req);
-    }
-    
-    if (ResourceRequest.ANY.equals(resourceName)) {
-      resourceRequestInfo.remoteRequest.setNodeLabelExpression(labelExpression);
-    }
+  private void addResourceRequest(Priority priority, String resourceName,
+      ExecutionTypeRequest execTypeReq, Resource capability, T req,
+      boolean relaxLocality, String labelExpression) {
+    @SuppressWarnings("unchecked")
+    ResourceRequestInfo resourceRequestInfo = remoteRequestsTable
+        .addResourceRequest(priority, resourceName,
+        execTypeReq, capability, req, relaxLocality, labelExpression);
 
     // Note this down for next interaction with ResourceManager
     addResourceRequestToAsk(resourceRequestInfo.remoteRequest);
@@ -800,70 +759,31 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
     }
   }
 
-  private void decResourceRequest(Priority priority, 
-                                   String resourceName,
-                                   Resource capability, 
-                                   T req) {
-    Map<String, TreeMap<Resource, ResourceRequestInfo>> remoteRequests =
-      this.remoteRequestsTable.get(priority);
-    
-    if(remoteRequests == null) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Not decrementing resource as priority " + priority 
-            + " is not present in request table");
-      }
-      return;
-    }
-    
-    Map<Resource, ResourceRequestInfo> reqMap = remoteRequests.get(resourceName);
-    if (reqMap == null) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Not decrementing resource as " + resourceName
-            + " is not present in request table");
-      }
-      return;
-    }
-    ResourceRequestInfo resourceRequestInfo = reqMap.get(capability);
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("BEFORE decResourceRequest:" + " applicationId="
-          + " priority=" + priority.getPriority()
-          + " resourceName=" + resourceName + " numContainers="
-          + resourceRequestInfo.remoteRequest.getNumContainers() 
-          + " #asks=" + ask.size());
-    }
-
-    resourceRequestInfo.remoteRequest.setNumContainers(
-        resourceRequestInfo.remoteRequest.getNumContainers() - 1);
-
-    resourceRequestInfo.containerRequests.remove(req);
-    
-    if(resourceRequestInfo.remoteRequest.getNumContainers() < 0) {
-      // guard against spurious removals
-      resourceRequestInfo.remoteRequest.setNumContainers(0);
-    }
+  private void decResourceRequest(Priority priority, String resourceName,
+      ExecutionTypeRequest execTypeReq, Resource capability, T req) {
+    @SuppressWarnings("unchecked")
+    ResourceRequestInfo resourceRequestInfo =
+        remoteRequestsTable.decResourceRequest(priority, resourceName,
+            execTypeReq, capability, req);
     // send the ResourceRequest to RM even if is 0 because it needs to override
     // a previously sent value. If ResourceRequest was not sent previously then
     // sending 0 aught to be a no-op on RM
-    addResourceRequestToAsk(resourceRequestInfo.remoteRequest);
+    if (resourceRequestInfo != null) {
+      addResourceRequestToAsk(resourceRequestInfo.remoteRequest);
 
-    // delete entries from map if no longer needed
-    if (resourceRequestInfo.remoteRequest.getNumContainers() == 0) {
-      reqMap.remove(capability);
-      if (reqMap.size() == 0) {
-        remoteRequests.remove(resourceName);
+      // delete entry from map if no longer needed
+      if (resourceRequestInfo.remoteRequest.getNumContainers() == 0) {
+        this.remoteRequestsTable.remove(priority, resourceName,
+            execTypeReq.getExecutionType(), capability);
       }
-      if (remoteRequests.size() == 0) {
-        remoteRequestsTable.remove(priority);
-      }
-    }
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("AFTER decResourceRequest:" + " applicationId="
-          + " priority=" + priority.getPriority()
-          + " resourceName=" + resourceName + " numContainers="
-          + resourceRequestInfo.remoteRequest.getNumContainers() 
-          + " #asks=" + ask.size());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("AFTER decResourceRequest:" + " applicationId="
+            + " priority=" + priority.getPriority()
+            + " resourceName=" + resourceName + " numContainers="
+            + resourceRequestInfo.remoteRequest.getNumContainers()
+            + " #asks=" + ask.size());
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51432779/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/RemoteRequestsTable.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/RemoteRequestsTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/RemoteRequestsTable.java
new file mode 100644
index 0000000..853a512
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/RemoteRequestsTable.java
@@ -0,0 +1,332 @@
+/**
+ * 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 org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl.ResourceRequestInfo;
+import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl.ResourceReverseMemoryThenCpuComparator;
+
+class RemoteRequestsTable<T> implements Iterable<ResourceRequestInfo>{
+
+  private static final Log LOG = LogFactory.getLog(RemoteRequestsTable.class);
+
+  static ResourceReverseMemoryThenCpuComparator resourceComparator =
+      new ResourceReverseMemoryThenCpuComparator();
+
+  /**
+   * Nested Iterator that iterates over just the ResourceRequestInfo
+   * object.
+   */
+  class RequestInfoIterator implements Iterator<ResourceRequestInfo> {
+    private Iterator<Map<String, Map<ExecutionType, TreeMap<Resource,
+        ResourceRequestInfo>>>> iLocMap;
+    private Iterator<Map<ExecutionType, TreeMap<Resource,
+        ResourceRequestInfo>>> iExecTypeMap;
+    private Iterator<TreeMap<Resource, ResourceRequestInfo>> iCapMap;
+    private Iterator<ResourceRequestInfo> iResReqInfo;
+
+    public RequestInfoIterator(Iterator<Map<String,
+        Map<ExecutionType, TreeMap<Resource, ResourceRequestInfo>>>>
+        iLocationMap) {
+      this.iLocMap = iLocationMap;
+      if (iLocMap.hasNext()) {
+        iExecTypeMap = iLocMap.next().values().iterator();
+      } else {
+        iExecTypeMap =
+            new LinkedList<Map<ExecutionType, TreeMap<Resource,
+                ResourceRequestInfo>>>().iterator();
+      }
+      if (iExecTypeMap.hasNext()) {
+        iCapMap = iExecTypeMap.next().values().iterator();
+      } else {
+        iCapMap =
+            new LinkedList<TreeMap<Resource, ResourceRequestInfo>>()
+                .iterator();
+      }
+      if (iCapMap.hasNext()) {
+        iResReqInfo = iCapMap.next().values().iterator();
+      } else {
+        iResReqInfo = new LinkedList<ResourceRequestInfo>().iterator();
+      }
+    }
+
+    @Override
+    public boolean hasNext() {
+      return iLocMap.hasNext()
+          || iExecTypeMap.hasNext()
+          || iCapMap.hasNext()
+          || iResReqInfo.hasNext();
+    }
+
+    @Override
+    public ResourceRequestInfo next() {
+      if (!iResReqInfo.hasNext()) {
+        if (!iCapMap.hasNext()) {
+          if (!iExecTypeMap.hasNext()) {
+            iExecTypeMap = iLocMap.next().values().iterator();
+          }
+          iCapMap = iExecTypeMap.next().values().iterator();
+        }
+        iResReqInfo = iCapMap.next().values().iterator();
+      }
+      return iResReqInfo.next();
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException("Remove is not supported" +
+          "for this iterator !!");
+    }
+  }
+
+  // Nest map with Primary key :
+  // Priority -> ResourceName(String) -> ExecutionType -> Capability(Resource)
+  // and value : ResourceRequestInfo
+  private Map<Priority, Map<String, Map<ExecutionType, TreeMap<Resource,
+      ResourceRequestInfo>>>> remoteRequestsTable = new HashMap<>();
+
+  @Override
+  public Iterator<ResourceRequestInfo> iterator() {
+    return new RequestInfoIterator(remoteRequestsTable.values().iterator());
+  }
+
+  ResourceRequestInfo get(Priority priority, String location,
+      ExecutionType execType, Resource capability) {
+    TreeMap<Resource, ResourceRequestInfo> capabilityMap =
+        getCapabilityMap(priority, location, execType);
+    if (capabilityMap == null) {
+      return null;
+    }
+    return capabilityMap.get(capability);
+  }
+
+  void put(Priority priority, String resourceName, ExecutionType execType,
+      Resource capability, ResourceRequestInfo resReqInfo) {
+    Map<String, Map<ExecutionType, TreeMap<Resource,
+        ResourceRequestInfo>>> locationMap =
+        remoteRequestsTable.get(priority);
+    if (locationMap == null) {
+      locationMap = new HashMap<>();
+      this.remoteRequestsTable.put(priority, locationMap);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Added priority=" + priority);
+      }
+    }
+    Map<ExecutionType, TreeMap<Resource, ResourceRequestInfo>> execTypeMap =
+        locationMap.get(resourceName);
+    if (execTypeMap == null) {
+      execTypeMap = new HashMap<>();
+      locationMap.put(resourceName, execTypeMap);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Added resourceName=" + resourceName);
+      }
+    }
+    TreeMap<Resource, ResourceRequestInfo> capabilityMap =
+        execTypeMap.get(execType);
+    if (capabilityMap == null) {
+      capabilityMap = new TreeMap<>(resourceComparator);
+      execTypeMap.put(execType, capabilityMap);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Added Execution Type=" + execType);
+      }
+    }
+    capabilityMap.put(capability, resReqInfo);
+  }
+
+  ResourceRequestInfo remove(Priority priority, String resourceName,
+      ExecutionType execType, Resource capability) {
+    ResourceRequestInfo retVal = null;
+    Map<String, Map<ExecutionType, TreeMap<Resource,
+        ResourceRequestInfo>>> locationMap = remoteRequestsTable.get(priority);
+    if (locationMap == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("No such priority=" + priority);
+      }
+      return null;
+    }
+    Map<ExecutionType, TreeMap<Resource, ResourceRequestInfo>>
+        execTypeMap = locationMap.get(resourceName);
+    if (execTypeMap == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("No such resourceName=" + resourceName);
+      }
+      return null;
+    }
+    TreeMap<Resource, ResourceRequestInfo> capabilityMap =
+        execTypeMap.get(execType);
+    if (capabilityMap == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("No such Execution Type=" + execType);
+      }
+      return null;
+    }
+    retVal = capabilityMap.remove(capability);
+    if (capabilityMap.size() == 0) {
+      execTypeMap.remove(execType);
+      if (execTypeMap.size() == 0) {
+        locationMap.remove(resourceName);
+        if (locationMap.size() == 0) {
+          this.remoteRequestsTable.remove(priority);
+        }
+      }
+    }
+    return retVal;
+  }
+
+  Map<String, Map<ExecutionType, TreeMap<Resource,
+      ResourceRequestInfo>>> getLocationMap(Priority priority) {
+    return remoteRequestsTable.get(priority);
+  }
+
+  Map<ExecutionType, TreeMap<Resource, ResourceRequestInfo>>
+      getExecutionTypeMap(Priority priority, String location) {
+    Map<String, Map<ExecutionType, TreeMap<Resource,
+        ResourceRequestInfo>>> locationMap = getLocationMap(priority);
+    if (locationMap == null) {
+      return null;
+    }
+    return locationMap.get(location);
+  }
+
+  TreeMap<Resource, ResourceRequestInfo> getCapabilityMap(Priority
+      priority, String location,
+      ExecutionType execType) {
+    Map<ExecutionType, TreeMap<Resource, ResourceRequestInfo>>
+        executionTypeMap = getExecutionTypeMap(priority, location);
+    if (executionTypeMap == null) {
+      return null;
+    }
+    return executionTypeMap.get(execType);
+  }
+
+  @SuppressWarnings("unchecked")
+  List<ResourceRequestInfo> getAllResourceRequestInfos(Priority priority,
+      Collection<String> locations) {
+    List retList = new LinkedList<>();
+    for (String location : locations) {
+      for (ExecutionType eType : ExecutionType.values()) {
+        TreeMap<Resource, ResourceRequestInfo> capabilityMap =
+            getCapabilityMap(priority, location, eType);
+        if (capabilityMap != null) {
+          retList.addAll(capabilityMap.values());
+        }
+      }
+    }
+    return retList;
+  }
+
+  List<ResourceRequestInfo> getMatchingRequests(
+      Priority priority, String resourceName, ExecutionType executionType,
+      Resource capability) {
+    List<ResourceRequestInfo> list = new LinkedList<>();
+    TreeMap<Resource, ResourceRequestInfo> capabilityMap =
+        getCapabilityMap(priority, resourceName, executionType);
+    if (capabilityMap != null) {
+      ResourceRequestInfo resourceRequestInfo = capabilityMap.get(capability);
+      if (resourceRequestInfo != null) {
+        list.add(resourceRequestInfo);
+      } else {
+        list.addAll(capabilityMap.tailMap(capability).values());
+      }
+    }
+    return list;
+  }
+
+  @SuppressWarnings("unchecked")
+  ResourceRequestInfo addResourceRequest(Priority priority, String resourceName,
+      ExecutionTypeRequest execTypeReq, Resource capability, T req,
+      boolean relaxLocality, String labelExpression) {
+    ResourceRequestInfo resourceRequestInfo = get(priority, resourceName,
+        execTypeReq.getExecutionType(), capability);
+    if (resourceRequestInfo == null) {
+      resourceRequestInfo =
+          new ResourceRequestInfo(priority, resourceName, capability,
+              relaxLocality);
+      put(priority, resourceName, execTypeReq.getExecutionType(), capability,
+          resourceRequestInfo);
+    }
+    resourceRequestInfo.remoteRequest.setExecutionTypeRequest(execTypeReq);
+    resourceRequestInfo.remoteRequest.setNumContainers(
+        resourceRequestInfo.remoteRequest.getNumContainers() + 1);
+
+    if (relaxLocality) {
+      resourceRequestInfo.containerRequests.add(req);
+    }
+
+    if (ResourceRequest.ANY.equals(resourceName)) {
+      resourceRequestInfo.remoteRequest.setNodeLabelExpression(labelExpression);
+    }
+    return resourceRequestInfo;
+  }
+
+  ResourceRequestInfo decResourceRequest(Priority priority, String resourceName,
+      ExecutionTypeRequest execTypeReq, Resource capability, T req) {
+    ResourceRequestInfo resourceRequestInfo = get(priority, resourceName,
+        execTypeReq.getExecutionType(), capability);
+
+    if (resourceRequestInfo == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Not decrementing resource as ResourceRequestInfo with" +
+            "priority=" + priority + ", " +
+            "resourceName=" + resourceName + ", " +
+            "executionType=" + execTypeReq + ", " +
+            "capability=" + capability + " is not present in request table");
+      }
+      return null;
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("BEFORE decResourceRequest:" + " applicationId="
+          + " priority=" + priority.getPriority()
+          + " resourceName=" + resourceName + " numContainers="
+          + resourceRequestInfo.remoteRequest.getNumContainers());
+    }
+
+    resourceRequestInfo.remoteRequest.setNumContainers(
+        resourceRequestInfo.remoteRequest.getNumContainers() - 1);
+
+    resourceRequestInfo.containerRequests.remove(req);
+
+    if (resourceRequestInfo.remoteRequest.getNumContainers() < 0) {
+      // guard against spurious removals
+      resourceRequestInfo.remoteRequest.setNumContainers(0);
+    }
+    return resourceRequestInfo;
+  }
+
+  boolean isEmpty() {
+    return remoteRequestsTable.isEmpty();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51432779/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/BaseAMRMProxyE2ETest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/BaseAMRMProxyE2ETest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/BaseAMRMProxyE2ETest.java
new file mode 100644
index 0000000..0b62054
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/BaseAMRMProxyE2ETest.java
@@ -0,0 +1,197 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.ClientRMProxy;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
+import org.apache.hadoop.yarn.server.MiniYARNCluster;
+import org.apache.hadoop.yarn.server.nodemanager.amrmproxy.AMRMProxyTokenSecretManager;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+
+/**
+ * Base test case to be used for Testing frameworks that use AMRMProxy.
+ */
+public abstract class BaseAMRMProxyE2ETest {
+
+  protected ApplicationMasterProtocol createAMRMProtocol(YarnClient rmClient,
+      ApplicationId appId, MiniYARNCluster cluster,
+      final Configuration yarnConf)
+      throws IOException, InterruptedException, YarnException {
+
+    UserGroupInformation user = null;
+
+    // Get the AMRMToken from AMRMProxy
+
+    ApplicationReport report = rmClient.getApplicationReport(appId);
+
+    user = UserGroupInformation.createProxyUser(
+        report.getCurrentApplicationAttemptId().toString(),
+        UserGroupInformation.getCurrentUser());
+
+    ContainerManagerImpl containerManager = (ContainerManagerImpl) cluster
+        .getNodeManager(0).getNMContext().getContainerManager();
+
+    AMRMProxyTokenSecretManager amrmTokenSecretManager =
+        containerManager.getAMRMProxyService().getSecretManager();
+    org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> token =
+        amrmTokenSecretManager
+            .createAndGetAMRMToken(report.getCurrentApplicationAttemptId());
+
+    SecurityUtil.setTokenService(token,
+        containerManager.getAMRMProxyService().getBindAddress());
+    user.addToken(token);
+
+    // Start Application Master
+
+    return user
+        .doAs(new PrivilegedExceptionAction<ApplicationMasterProtocol>() {
+          @Override
+          public ApplicationMasterProtocol run() throws Exception {
+            return ClientRMProxy.createRMProxy(yarnConf,
+                ApplicationMasterProtocol.class);
+          }
+        });
+  }
+
+  protected AllocateRequest createAllocateRequest(List<NodeReport> listNode) {
+    // The test needs AMRMClient to create a real allocate request
+    AMRMClientImpl<AMRMClient.ContainerRequest> amClient =
+        new AMRMClientImpl<>();
+
+    Resource capability = Resource.newInstance(1024, 2);
+    Priority priority = Priority.newInstance(1);
+    List<NodeReport> nodeReports = listNode;
+    String node = nodeReports.get(0).getNodeId().getHost();
+    String[] nodes = new String[] {node};
+
+    AMRMClient.ContainerRequest storedContainer1 =
+        new AMRMClient.ContainerRequest(capability, nodes, null, priority);
+    amClient.addContainerRequest(storedContainer1);
+    amClient.addContainerRequest(storedContainer1);
+
+    List<ResourceRequest> resourceAsk = new ArrayList<>();
+    for (ResourceRequest rr : amClient.ask) {
+      resourceAsk.add(rr);
+    }
+
+    ResourceBlacklistRequest resourceBlacklistRequest = ResourceBlacklistRequest
+        .newInstance(new ArrayList<>(), new ArrayList<>());
+
+    int responseId = 1;
+
+    return AllocateRequest.newInstance(responseId, 0, resourceAsk,
+        new ArrayList<>(), resourceBlacklistRequest);
+  }
+
+  protected ApplicationAttemptId createApp(YarnClient yarnClient,
+      MiniYARNCluster yarnCluster, Configuration conf) throws Exception {
+
+    ApplicationSubmissionContext appContext =
+        yarnClient.createApplication().getApplicationSubmissionContext();
+    ApplicationId appId = appContext.getApplicationId();
+
+    appContext.setApplicationName("Test");
+
+    Priority pri = Records.newRecord(Priority.class);
+    pri.setPriority(0);
+    appContext.setPriority(pri);
+
+    appContext.setQueue("default");
+
+    ContainerLaunchContext amContainer = BuilderUtils.newContainerLaunchContext(
+        Collections.<String, LocalResource> emptyMap(),
+        new HashMap<String, String>(), Arrays.asList("sleep", "10000"),
+        new HashMap<String, ByteBuffer>(), null,
+        new HashMap<ApplicationAccessType, String>());
+    appContext.setAMContainerSpec(amContainer);
+    appContext.setResource(Resource.newInstance(1024, 1));
+
+    SubmitApplicationRequest appRequest =
+        Records.newRecord(SubmitApplicationRequest.class);
+    appRequest.setApplicationSubmissionContext(appContext);
+
+    yarnClient.submitApplication(appContext);
+
+    RMAppAttempt appAttempt = null;
+    ApplicationAttemptId attemptId = null;
+    while (true) {
+      ApplicationReport appReport = yarnClient.getApplicationReport(appId);
+      if (appReport
+          .getYarnApplicationState() == YarnApplicationState.ACCEPTED) {
+        attemptId =
+            appReport.getCurrentApplicationAttemptId();
+        appAttempt = yarnCluster.getResourceManager().getRMContext().getRMApps()
+            .get(attemptId.getApplicationId()).getCurrentAppAttempt();
+        while (true) {
+          if (appAttempt.getAppAttemptState() == RMAppAttemptState.LAUNCHED) {
+            break;
+          }
+        }
+        break;
+      }
+    }
+    Thread.sleep(1000);
+    // Just dig into the ResourceManager and get the AMRMToken just for the sake
+    // of testing.
+    UserGroupInformation.setLoginUser(UserGroupInformation
+        .createRemoteUser(UserGroupInformation.getCurrentUser().getUserName()));
+
+    // emulate RM setup of AMRM token in credentials by adding the token
+    // *before* setting the token service
+    UserGroupInformation.getCurrentUser().addToken(appAttempt.getAMRMToken());
+    appAttempt.getAMRMToken().setService(
+        ClientRMProxy.getAMRMTokenService(conf));
+    return attemptId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51432779/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
index 75b49d0..99bfca5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
@@ -61,6 +61,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.NMToken;
@@ -413,11 +414,13 @@ public class TestAMRMClient {
       amClient.addContainerRequest(storedContainer3);
       
       // test addition and storage
-      int containersRequestedAny = amClient.remoteRequestsTable.get(priority)
-       .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers();
+      int containersRequestedAny = amClient.remoteRequestsTable.get(priority,
+          ResourceRequest.ANY, ExecutionType.GUARANTEED, capability)
+          .remoteRequest.getNumContainers();
       assertEquals(2, containersRequestedAny);
-      containersRequestedAny = amClient.remoteRequestsTable.get(priority1)
-          .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers();
+      containersRequestedAny = amClient.remoteRequestsTable.get(priority1,
+          ResourceRequest.ANY, ExecutionType.GUARANTEED, capability)
+          .remoteRequest.getNumContainers();
          assertEquals(1, containersRequestedAny);
       List<? extends Collection<ContainerRequest>> matches = 
           amClient.getMatchingRequests(priority, node, capability);
@@ -919,12 +922,15 @@ public class TestAMRMClient {
     amClient.removeContainerRequest(
         new ContainerRequest(capability, nodes, racks, priority));
     
-    int containersRequestedNode = amClient.remoteRequestsTable.get(priority)
-        .get(node).get(capability).remoteRequest.getNumContainers();
-    int containersRequestedRack = amClient.remoteRequestsTable.get(priority)
-        .get(rack).get(capability).remoteRequest.getNumContainers();
-    int containersRequestedAny = amClient.remoteRequestsTable.get(priority)
-    .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers();
+    int containersRequestedNode = amClient.remoteRequestsTable.get(priority,
+        node, ExecutionType.GUARANTEED, capability).remoteRequest
+        .getNumContainers();
+    int containersRequestedRack = amClient.remoteRequestsTable.get(priority,
+        rack, ExecutionType.GUARANTEED, capability).remoteRequest
+        .getNumContainers();
+    int containersRequestedAny = amClient.remoteRequestsTable.get(priority,
+        ResourceRequest.ANY, ExecutionType.GUARANTEED, capability)
+        .remoteRequest.getNumContainers();
 
     assertEquals(2, containersRequestedNode);
     assertEquals(2, containersRequestedRack);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51432779/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientContainerRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientContainerRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientContainerRequest.java
index cb8c86a..2db33c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientContainerRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientContainerRequest.java
@@ -26,6 +26,8 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.net.DNSToSwitchMapping;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -35,6 +37,46 @@ import org.apache.hadoop.yarn.client.api.InvalidContainerRequestException;
 import org.junit.Test;
 
 public class TestAMRMClientContainerRequest {
+
+  @Test
+  public void testOpportunisticAndGuaranteedRequests() {
+    AMRMClientImpl<ContainerRequest> client =
+        new AMRMClientImpl<ContainerRequest>();
+
+    Configuration conf = new Configuration();
+    conf.setClass(
+        CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+        MyResolver.class, DNSToSwitchMapping.class);
+    client.init(conf);
+
+    Resource capability = Resource.newInstance(1024, 1);
+    ContainerRequest request =
+        new ContainerRequest(capability, new String[] {"host1", "host2"},
+            new String[] {"/rack2"}, Priority.newInstance(1));
+    client.addContainerRequest(request);
+    verifyResourceRequest(client, request, "host1", true);
+    verifyResourceRequest(client, request, "host2", true);
+    verifyResourceRequest(client, request, "/rack1", true);
+    verifyResourceRequest(client, request, "/rack2", true);
+    verifyResourceRequest(client, request, ResourceRequest.ANY, true);
+    ContainerRequest request2 =
+        new ContainerRequest(capability, new String[] {"host1", "host2"},
+            new String[] {"/rack2"}, Priority.newInstance(1), true, null,
+            ExecutionTypeRequest.newInstance(
+                ExecutionType.OPPORTUNISTIC, true));
+    client.addContainerRequest(request2);
+    verifyResourceRequest(client, request, "host1", true,
+        ExecutionType.OPPORTUNISTIC);
+    verifyResourceRequest(client, request, "host2", true,
+        ExecutionType.OPPORTUNISTIC);
+    verifyResourceRequest(client, request, "/rack1", true,
+        ExecutionType.OPPORTUNISTIC);
+    verifyResourceRequest(client, request, "/rack2", true,
+        ExecutionType.OPPORTUNISTIC);
+    verifyResourceRequest(client, request, ResourceRequest.ANY, true,
+        ExecutionType.OPPORTUNISTIC);
+  }
+
   @Test
   public void testFillInRacks() {
     AMRMClientImpl<ContainerRequest> client =
@@ -224,8 +266,16 @@ public class TestAMRMClientContainerRequest {
   private void verifyResourceRequest(
       AMRMClientImpl<ContainerRequest> client, ContainerRequest request,
       String location, boolean expectedRelaxLocality) {
-    ResourceRequest ask =  client.remoteRequestsTable.get(request.getPriority())
-        .get(location).get(request.getCapability()).remoteRequest;
+    verifyResourceRequest(client, request, location, expectedRelaxLocality,
+        ExecutionType.GUARANTEED);
+  }
+
+  private void verifyResourceRequest(
+      AMRMClientImpl<ContainerRequest> client, ContainerRequest request,
+      String location, boolean expectedRelaxLocality,
+      ExecutionType executionType) {
+    ResourceRequest ask = client.remoteRequestsTable.get(request.getPriority(),
+        location, executionType, request.getCapability()).remoteRequest;
     assertEquals(location, ask.getResourceName());
     assertEquals(1, ask.getNumContainers());
     assertEquals(expectedRelaxLocality, ask.getRelaxLocality());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51432779/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java
index f1e3f03..33f7527 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java
@@ -19,20 +19,12 @@
 package org.apache.hadoop.yarn.client.api.impl;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
@@ -40,43 +32,25 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
-import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.Token;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
-import org.apache.hadoop.yarn.client.ClientRMProxy;
-import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
-import org.apache.hadoop.yarn.server.nodemanager.amrmproxy.AMRMProxyTokenSecretManager;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
-import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.apache.hadoop.yarn.util.Records;
 import org.junit.Assert;
 import org.junit.Test;
 
-public class TestAMRMProxy {
+/**
+ * End-to-End test cases for the AMRMProxy Service.
+ */
+public class TestAMRMProxy extends BaseAMRMProxyE2ETest {
 
   private static final Log LOG = LogFactory.getLog(TestAMRMProxy.class);
 
@@ -84,7 +58,7 @@ public class TestAMRMProxy {
    * This test validates register, allocate and finish of an application through
    * the AMRMPRoxy.
    */
-  @Test(timeout = 60000)
+  @Test(timeout = 120000)
   public void testAMRMProxyE2E() throws Exception {
     MiniYARNCluster cluster = new MiniYARNCluster("testAMRMProxyE2E", 1, 1, 1);
     YarnClient rmClient = null;
@@ -107,7 +81,8 @@ public class TestAMRMProxy {
 
       // Submit application
 
-      ApplicationId appId = createApp(rmClient, cluster);
+      ApplicationAttemptId appAttmptId = createApp(rmClient, cluster, conf);
+      ApplicationId appId = appAttmptId.getApplicationId();
 
       client = createAMRMProtocol(rmClient, appId, cluster, yarnConf);
 
@@ -173,7 +148,7 @@ public class TestAMRMProxy {
    * that the received token it is different from the previous one within 5
    * requests.
    */
-  @Test(timeout = 60000)
+  @Test(timeout = 120000)
   public void testE2ETokenRenewal() throws Exception {
     MiniYARNCluster cluster =
         new MiniYARNCluster("testE2ETokenRenewal", 1, 1, 1);
@@ -201,7 +176,8 @@ public class TestAMRMProxy {
 
       // Submit
 
-      ApplicationId appId = createApp(rmClient, cluster);
+      ApplicationAttemptId appAttmptId = createApp(rmClient, cluster, conf);
+      ApplicationId appId = appAttmptId.getApplicationId();
 
       client = createAMRMProtocol(rmClient, appId, cluster, yarnConf);
 
@@ -252,7 +228,7 @@ public class TestAMRMProxy {
    * This test validates that an AM cannot register directly to the RM, with the
    * token provided by the AMRMProxy.
    */
-  @Test(timeout = 60000)
+  @Test(timeout = 120000)
   public void testE2ETokenSwap() throws Exception {
     MiniYARNCluster cluster = new MiniYARNCluster("testE2ETokenSwap", 1, 1, 1);
     YarnClient rmClient = null;
@@ -270,7 +246,8 @@ public class TestAMRMProxy {
       rmClient.init(yarnConf);
       rmClient.start();
 
-      ApplicationId appId = createApp(rmClient, cluster);
+      ApplicationAttemptId appAttmptId = createApp(rmClient, cluster, conf);
+      ApplicationId appId = appAttmptId.getApplicationId();
 
       client = createAMRMProtocol(rmClient, appId, cluster, yarnConf);
 
@@ -290,124 +267,4 @@ public class TestAMRMProxy {
       cluster.stop();
     }
   }
-
-  protected ApplicationMasterProtocol createAMRMProtocol(YarnClient rmClient,
-      ApplicationId appId, MiniYARNCluster cluster,
-      final Configuration yarnConf)
-          throws IOException, InterruptedException, YarnException {
-
-    UserGroupInformation user = null;
-
-    // Get the AMRMToken from AMRMProxy
-
-    ApplicationReport report = rmClient.getApplicationReport(appId);
-
-    user = UserGroupInformation.createProxyUser(
-        report.getCurrentApplicationAttemptId().toString(),
-        UserGroupInformation.getCurrentUser());
-
-    ContainerManagerImpl containerManager = (ContainerManagerImpl) cluster
-        .getNodeManager(0).getNMContext().getContainerManager();
-
-    AMRMProxyTokenSecretManager amrmTokenSecretManager =
-        containerManager.getAMRMProxyService().getSecretManager();
-    org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> token =
-        amrmTokenSecretManager
-            .createAndGetAMRMToken(report.getCurrentApplicationAttemptId());
-
-    SecurityUtil.setTokenService(token,
-        containerManager.getAMRMProxyService().getBindAddress());
-    user.addToken(token);
-
-    // Start Application Master
-
-    return user
-        .doAs(new PrivilegedExceptionAction<ApplicationMasterProtocol>() {
-          @Override
-          public ApplicationMasterProtocol run() throws Exception {
-            return ClientRMProxy.createRMProxy(yarnConf,
-                ApplicationMasterProtocol.class);
-          }
-        });
-  }
-
-  protected AllocateRequest createAllocateRequest(List<NodeReport> listNode) {
-    // The test needs AMRMClient to create a real allocate request
-    AMRMClientImpl<ContainerRequest> amClient =
-        new AMRMClientImpl<ContainerRequest>();
-
-    Resource capability = Resource.newInstance(1024, 2);
-    Priority priority = Priority.newInstance(1);
-    List<NodeReport> nodeReports = listNode;
-    String node = nodeReports.get(0).getNodeId().getHost();
-    String[] nodes = new String[] { node };
-
-    ContainerRequest storedContainer1 =
-        new ContainerRequest(capability, nodes, null, priority);
-    amClient.addContainerRequest(storedContainer1);
-    amClient.addContainerRequest(storedContainer1);
-
-    List<ResourceRequest> resourceAsk = new ArrayList<ResourceRequest>();
-    for (ResourceRequest rr : amClient.ask) {
-      resourceAsk.add(rr);
-    }
-
-    ResourceBlacklistRequest resourceBlacklistRequest = ResourceBlacklistRequest
-        .newInstance(new ArrayList<String>(), new ArrayList<String>());
-
-    int responseId = 1;
-
-    return AllocateRequest.newInstance(responseId, 0, resourceAsk,
-        new ArrayList<ContainerId>(), resourceBlacklistRequest);
-  }
-
-  protected ApplicationId createApp(YarnClient yarnClient,
-      MiniYARNCluster yarnCluster) throws Exception {
-
-    ApplicationSubmissionContext appContext =
-        yarnClient.createApplication().getApplicationSubmissionContext();
-    ApplicationId appId = appContext.getApplicationId();
-
-    appContext.setApplicationName("Test");
-
-    Priority pri = Records.newRecord(Priority.class);
-    pri.setPriority(0);
-    appContext.setPriority(pri);
-
-    appContext.setQueue("default");
-
-    ContainerLaunchContext amContainer = BuilderUtils.newContainerLaunchContext(
-        Collections.<String, LocalResource> emptyMap(),
-        new HashMap<String, String>(), Arrays.asList("sleep", "10000"),
-        new HashMap<String, ByteBuffer>(), null,
-        new HashMap<ApplicationAccessType, String>());
-    appContext.setAMContainerSpec(amContainer);
-    appContext.setResource(Resource.newInstance(1024, 1));
-
-    SubmitApplicationRequest appRequest =
-        Records.newRecord(SubmitApplicationRequest.class);
-    appRequest.setApplicationSubmissionContext(appContext);
-
-    yarnClient.submitApplication(appContext);
-
-    RMAppAttempt appAttempt = null;
-    while (true) {
-      ApplicationReport appReport = yarnClient.getApplicationReport(appId);
-      if (appReport
-          .getYarnApplicationState() == YarnApplicationState.ACCEPTED) {
-        ApplicationAttemptId attemptId =
-            appReport.getCurrentApplicationAttemptId();
-        appAttempt = yarnCluster.getResourceManager().getRMContext().getRMApps()
-            .get(attemptId.getApplicationId()).getCurrentAppAttempt();
-        while (true) {
-          if (appAttempt.getAppAttemptState() == RMAppAttemptState.LAUNCHED) {
-            break;
-          }
-        }
-        break;
-      }
-    }
-    Thread.sleep(1000);
-    return appId;
-  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[51/51] [abbrv] hadoop git commit: YARN-4830. Add support for resource types in the nodemanager. Contributed by Varun Vasudev.

Posted by vv...@apache.org.
YARN-4830. Add support for resource types in the nodemanager. Contributed by Varun Vasudev.


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

Branch: refs/heads/YARN-3926
Commit: 7a0d5db2d7aba647c6eb9a7a9847bae4458ccb0d
Parents: dd046eb
Author: Varun Vasudev <vv...@apache.org>
Authored: Sat Jun 11 14:33:46 2016 +0530
Committer: Varun Vasudev <vv...@apache.org>
Committed: Mon Jun 13 14:47:03 2016 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/api/records/Resource.java       |   3 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |  19 ++-
 .../FileSystemBasedConfigurationProvider.java   |   3 +-
 .../hadoop/yarn/LocalConfigurationProvider.java |   3 +-
 .../api/records/impl/pb/ResourcePBImpl.java     |  53 +++---
 .../yarn/util/resource/ResourceUtils.java       | 168 +++++++++++++++----
 .../yarn/util/resource/TestResourceUtils.java   |  29 +++-
 .../resource-types/node-resources-1.xml         |  29 ++++
 .../resource-types/node-resources-2.xml         |  39 +++++
 .../nodemanager/NodeStatusUpdaterImpl.java      |   7 +-
 .../util/NodeManagerHardwareUtils.java          |  52 ++++++
 .../resourcemanager/ResourceTrackerService.java |   9 +-
 12 files changed, 342 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a0d5db2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
index 21d1aed..af9ce1d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
@@ -254,7 +254,8 @@ public abstract class Resource implements Comparable<Resource> {
         continue;
       }
       if (entry.getKey().equals(ResourceInformation.VCORES.getName())
-          && entry.getValue().getUnits().equals("")) {
+          && entry.getValue().getUnits()
+          .equals(ResourceInformation.VCORES.getUnits())) {
         continue;
       }
       sb.append(", ").append(entry.getKey()).append(": ")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a0d5db2/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 a73d541..52d4ab8 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
@@ -62,6 +62,10 @@ public class YarnConfiguration extends Configuration {
       "resource-types.xml";
 
   @Private
+  public static final String NODE_RESOURCES_CONFIGURATION_FILE =
+      "node-resources.xml";
+
+  @Private
   public static final List<String> RM_CONFIGURATION_FILES =
       Collections.unmodifiableList(Arrays.asList(
           RESOURCE_TYPES_CONFIGURATION_FILE,
@@ -71,6 +75,16 @@ public class YarnConfiguration extends Configuration {
           YARN_SITE_CONFIGURATION_FILE,
           CORE_SITE_CONFIGURATION_FILE));
 
+  @Private
+  public static final List<String> NM_CONFIGURATION_FILES =
+      Collections.unmodifiableList(Arrays.asList(
+          NODE_RESOURCES_CONFIGURATION_FILE,
+          DR_CONFIGURATION_FILE,
+          CS_CONFIGURATION_FILE,
+          HADOOP_POLICY_CONFIGURATION_FILE,
+          YARN_SITE_CONFIGURATION_FILE,
+          CORE_SITE_CONFIGURATION_FILE));
+
   @Evolving
   public static final int APPLICATION_MAX_TAGS = 10;
 
@@ -95,12 +109,15 @@ public class YarnConfiguration extends Configuration {
   public static final String YARN_PREFIX = "yarn.";
 
   /////////////////////////////
-  // Scheduler resource types configs
+  // Resource types configs
   ////////////////////////////
 
   public static final String RESOURCE_TYPES =
       YarnConfiguration.YARN_PREFIX + "resource-types";
 
+  public static final String NM_RESOURCES_PREFIX =
+      YarnConfiguration.NM_PREFIX + "resource-type.";
+
   /** Delay before deleting resource to ease debugging of NM issues */
   public static final String DEBUG_NM_DELETE_DELAY_SEC =
     YarnConfiguration.NM_PREFIX + "delete.debug-delay-sec";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a0d5db2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/FileSystemBasedConfigurationProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/FileSystemBasedConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/FileSystemBasedConfigurationProvider.java
index bf50cad..227c94f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/FileSystemBasedConfigurationProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/FileSystemBasedConfigurationProvider.java
@@ -51,7 +51,8 @@ public class FileSystemBasedConfigurationProvider
           "Illegal argument! The parameter should not be null or empty");
     }
     Path filePath;
-    if (YarnConfiguration.RM_CONFIGURATION_FILES.contains(name)) {
+    if (YarnConfiguration.RM_CONFIGURATION_FILES.contains(name) ||
+        YarnConfiguration.NM_CONFIGURATION_FILES.contains(name)) {
       filePath = new Path(this.configDir, name);
       if (!fs.exists(filePath)) {
         LOG.info(filePath + " not found");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a0d5db2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/LocalConfigurationProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/LocalConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/LocalConfigurationProvider.java
index cfa194f..0cdbd15 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/LocalConfigurationProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/LocalConfigurationProvider.java
@@ -39,7 +39,8 @@ public class LocalConfigurationProvider extends ConfigurationProvider {
     if (name == null || name.isEmpty()) {
       throw new YarnException(
           "Illegal argument! The parameter should not be null or empty");
-    } else if (YarnConfiguration.RM_CONFIGURATION_FILES.contains(name)) {
+    } else if (YarnConfiguration.RM_CONFIGURATION_FILES.contains(name) ||
+        YarnConfiguration.NM_CONFIGURATION_FILES.contains(name)) {
       return bootstrapConf.getConfResourceAsInputStream(name);
     }
     return new FileInputStream(name);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a0d5db2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
index f4ca882..1327191 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
@@ -18,7 +18,8 @@
 
 package org.apache.hadoop.yarn.api.records.impl.pb;
 
-
+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.yarn.api.protocolrecords.ResourceTypes;
@@ -38,6 +39,8 @@ import java.util.*;
 @Unstable
 public class ResourcePBImpl extends Resource {
 
+  private static final Log LOG = LogFactory.getLog(ResourcePBImpl.class);
+
   ResourceProto proto = ResourceProto.getDefaultInstance();
   ResourceProto.Builder builder = null;
   boolean viaProto = false;
@@ -78,10 +81,12 @@ public class ResourcePBImpl extends Resource {
   @Override
   public long getMemorySize() {
     // memory should always be present
-    initResourcesMap();
+    initResources();
     ResourceInformation ri =
         this.getResourceInformation(ResourceInformation.MEMORY_MB.getName());
-    return UnitsConversionUtil.convert(ri.getUnits(), "Mi", ri.getValue());
+    return UnitsConversionUtil
+        .convert(ri.getUnits(), ResourceInformation.MEMORY_MB.getUnits(),
+            ri.getValue());
   }
 
   @Override
@@ -101,20 +106,15 @@ public class ResourcePBImpl extends Resource {
   @Override
   public long getVirtualCoresSize() {
     // vcores should always be present
-    initResourcesMap();
+    initResources();
     return this.getResourceValue(ResourceInformation.VCORES.getName());
   }
 
   @Override
   public void setVirtualCores(long vCores) {
-    try {
-      setResourceValue(ResourceInformation.VCORES.getName(),
-          Long.valueOf(vCores));
-    } catch (ResourceNotFoundException re) {
-      this.setResourceInformation(ResourceInformation.VCORES.getName(),
-          ResourceInformation.newInstance(ResourceInformation.VCORES.getName(),
-              vCores));
-    }
+    setResourceInformation(ResourceInformation.VCORES.getName(),
+        ResourceInformation.newInstance(ResourceInformation.VCORES.getName(),
+            ResourceInformation.VCORES.getUnits(), (long) vCores));
   }
 
   private void initResources() {
@@ -131,14 +131,16 @@ public class ResourcePBImpl extends Resource {
       Long value = entry.hasValue() ? entry.getValue() : 0L;
       ResourceInformation ri =
           ResourceInformation.newInstance(entry.getKey(), units, value, type);
-      resources.put(ri.getName(), ri);
-    }
-    if(this.getMemory() != p.getMemory()) {
-      setMemory(p.getMemory());
-    }
-    if(this.getVirtualCores() != p.getVirtualCores()) {
-      setVirtualCores(p.getVirtualCores());
+      if (resources.containsKey(ri.getName())) {
+        resources.get(ri.getName()).setResourceType(ri.getResourceType());
+        resources.get(ri.getName()).setUnits(ri.getUnits());
+        resources.get(ri.getName()).setValue(value);
+      } else {
+        LOG.warn("Got unknown resource type: " + ri.getName() + "; skipping");
+      }
     }
+    this.setMemory(p.getMemory());
+    this.setVirtualCores(p.getVirtualCores());
   }
 
   @Override
@@ -152,7 +154,7 @@ public class ResourcePBImpl extends Resource {
     if (!resource.equals(resourceInformation.getName())) {
       resourceInformation.setName(resource);
     }
-    initResourcesMap();
+    initResources();
     resources.put(resource, resourceInformation);
   }
 
@@ -160,6 +162,7 @@ public class ResourcePBImpl extends Resource {
   public void setResourceValue(String resource, Long value)
       throws ResourceNotFoundException {
     maybeInitBuilder();
+    initResources();
     if (resource == null) {
       throw new IllegalArgumentException("resource type object cannot be null");
     }
@@ -167,9 +170,7 @@ public class ResourcePBImpl extends Resource {
       throw new ResourceNotFoundException(
           "Resource " + resource + " not found");
     }
-    ResourceInformation ri = resources.get(resource);
-    ri.setValue(value);
-    resources.put(resource, ri);
+    resources.get(resource).setValue(value);
   }
 
   @Override
@@ -214,8 +215,10 @@ public class ResourcePBImpl extends Resource {
   synchronized private void mergeLocalToBuilder() {
     builder.clearResourceValueMap();
     if (resources != null && !resources.isEmpty()) {
-      for (Map.Entry<String, ResourceInformation> entry : resources.entrySet()) {
-        ResourceInformationProto.Builder e = ResourceInformationProto.newBuilder();
+      for (Map.Entry<String, ResourceInformation> entry :
+          resources.entrySet()) {
+        ResourceInformationProto.Builder e =
+            ResourceInformationProto.newBuilder();
         e.setKey(entry.getKey());
         e.setUnits(entry.getValue().getUnits());
         e.setType(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a0d5db2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
index 0316f01..4f56fbb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.util.resource;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -51,15 +52,21 @@ public class ResourceUtils {
   public static final String UNITS = ".units";
   public static final String TYPE = ".type";
 
+  private static final String MEMORY = ResourceInformation.MEMORY_MB.getName();
+  private static final String VCORES = ResourceInformation.VCORES.getName();
+
   private static final Set<String> DISALLOWED_NAMES = new HashSet<>();
   static {
     DISALLOWED_NAMES.add("memory");
-    DISALLOWED_NAMES.add(ResourceInformation.MEMORY_MB.getName());
-    DISALLOWED_NAMES.add(ResourceInformation.VCORES.getName());
+    DISALLOWED_NAMES.add(MEMORY);
+    DISALLOWED_NAMES.add(VCORES);
   }
 
   private static volatile Object lock;
   private static Map<String, ResourceInformation> readOnlyResources;
+  private static volatile Object nodeLock;
+  private static Map<String, ResourceInformation> readOnlyNodeResources;
+
 
   static final Log LOG = LogFactory.getLog(ResourceUtils.class);
 
@@ -69,22 +76,20 @@ public class ResourceUtils {
   private static void checkMandatatoryResources(
       Map<String, ResourceInformation> resourceInformationMap)
       throws YarnRuntimeException {
-    String memory = ResourceInformation.MEMORY_MB.getName();
-    String vcores = ResourceInformation.VCORES.getName();
-    if (resourceInformationMap.containsKey(memory)) {
-      ResourceInformation memInfo = resourceInformationMap.get(memory);
+    if (resourceInformationMap.containsKey(MEMORY)) {
+      ResourceInformation memInfo = resourceInformationMap.get(MEMORY);
       String memUnits = ResourceInformation.MEMORY_MB.getUnits();
       ResourceTypes memType = ResourceInformation.MEMORY_MB.getResourceType();
       if (!memInfo.getUnits().equals(memUnits) || !memInfo.getResourceType()
           .equals(memType)) {
         throw new YarnRuntimeException(
             "Attempt to re-define mandatory resource 'memory-mb'. It can only"
-                + " be of type 'COUNTABLE' and have units 'M'.");
+                + " be of type 'COUNTABLE' and have units 'Mi'.");
       }
     }
 
-    if (resourceInformationMap.containsKey(vcores)) {
-      ResourceInformation vcoreInfo = resourceInformationMap.get(vcores);
+    if (resourceInformationMap.containsKey(VCORES)) {
+      ResourceInformation vcoreInfo = resourceInformationMap.get(VCORES);
       String vcoreUnits = ResourceInformation.VCORES.getUnits();
       ResourceTypes vcoreType = ResourceInformation.VCORES.getResourceType();
       if (!vcoreInfo.getUnits().equals(vcoreUnits) || !vcoreInfo
@@ -99,21 +104,21 @@ public class ResourceUtils {
   private static void addManadtoryResources(
       Map<String, ResourceInformation> res) {
     ResourceInformation ri;
-    if (!res.containsKey(ResourceInformation.MEMORY_MB.getName())) {
-      LOG.info("Adding resource type - name = " + ResourceInformation.MEMORY_MB
-          .getName() + ", units = " + ResourceInformation.MEMORY_MB.getUnits()
-          + ", type = " + ResourceTypes.COUNTABLE);
+    if (!res.containsKey(MEMORY)) {
+      LOG.info("Adding resource type - name = " + MEMORY + ", units = "
+          + ResourceInformation.MEMORY_MB.getUnits() + ", type = "
+          + ResourceTypes.COUNTABLE);
       ri = ResourceInformation
-          .newInstance(ResourceInformation.MEMORY_MB.getName(),
+          .newInstance(MEMORY,
               ResourceInformation.MEMORY_MB.getUnits());
-      res.put(ResourceInformation.MEMORY_MB.getName(), ri);
+      res.put(MEMORY, ri);
     }
-    if (!res.containsKey(ResourceInformation.VCORES.getName())) {
-      LOG.info("Adding resource type - name = " + ResourceInformation.VCORES
-          .getName() + ", units = , type = " + ResourceTypes.COUNTABLE);
+    if (!res.containsKey(VCORES)) {
+      LOG.info("Adding resource type - name = " + VCORES + ", units = , type = "
+          + ResourceTypes.COUNTABLE);
       ri =
-          ResourceInformation.newInstance(ResourceInformation.VCORES.getName());
-      res.put(ResourceInformation.VCORES.getName(), ri);
+          ResourceInformation.newInstance(VCORES);
+      res.put(VCORES, ri);
     }
   }
 
@@ -122,6 +127,7 @@ public class ResourceUtils {
       Map<String, ResourceInformation> resourceInformationMap) {
 
     String[] resourceNames = conf.getStrings(YarnConfiguration.RESOURCE_TYPES);
+
     if (resourceNames != null && resourceNames.length != 0) {
       for (String resourceName : resourceNames) {
         String resourceUnits = conf.get(
@@ -178,25 +184,13 @@ public class ResourceUtils {
               conf = new YarnConfiguration();
             }
             try {
-              InputStream ris = getConfInputStream(resourceFile, conf);
+              addResourcesFileToConf(resourceFile, conf);
               LOG.debug("Found " + resourceFile + ", adding to configuration");
-              conf.addResource(ris);
               initializeResourcesMap(conf, resources);
-              return resources;
             } catch (FileNotFoundException fe) {
               LOG.info("Unable to find '" + resourceFile
                   + "'. Falling back to memory and vcores as resources", fe);
               initializeResourcesMap(conf, resources);
-            } catch (IOException ie) {
-              LOG.fatal(
-                  "Exception trying to read resource types configuration '"
-                      + resourceFile + "'.", ie);
-              throw new YarnRuntimeException(ie);
-            } catch (YarnException ye) {
-              LOG.fatal(
-                  "YARN Exception trying to read resource types configuration '"
-                      + resourceFile + "'.", ye);
-              throw new YarnRuntimeException(ye);
             }
           }
         }
@@ -205,8 +199,8 @@ public class ResourceUtils {
     return readOnlyResources;
   }
 
-  static InputStream getConfInputStream(String resourceFile, Configuration conf)
-      throws IOException, YarnException {
+  private static InputStream getConfInputStream(String resourceFile,
+      Configuration conf) throws IOException, YarnException {
 
     ConfigurationProvider provider =
         ConfigurationProviderFactory.getConfigurationProvider(conf);
@@ -222,8 +216,112 @@ public class ResourceUtils {
     return ris;
   }
 
+  private static void addResourcesFileToConf(String resourceFile,
+      Configuration conf) throws FileNotFoundException {
+    try {
+      InputStream ris = getConfInputStream(resourceFile, conf);
+      LOG.debug("Found " + resourceFile + ", adding to configuration");
+      conf.addResource(ris);
+    } catch (FileNotFoundException fe) {
+      throw fe;
+    } catch (IOException ie) {
+      LOG.fatal("Exception trying to read resource types configuration '"
+          + resourceFile + "'.", ie);
+      throw new YarnRuntimeException(ie);
+    } catch (YarnException ye) {
+      LOG.fatal("YARN Exception trying to read resource types configuration '"
+          + resourceFile + "'.", ye);
+      throw new YarnRuntimeException(ye);
+    }
+  }
+
   @VisibleForTesting
   static void resetResourceTypes() {
     lock = null;
   }
+
+  private static String getUnits(String resourceValue) {
+    String units;
+    for (int i = 0; i < resourceValue.length(); i++) {
+      if (Character.isAlphabetic(resourceValue.charAt(i))) {
+        units = resourceValue.substring(i);
+        if (StringUtils.isAlpha(units)) {
+          return units;
+        }
+      }
+    }
+    return "";
+  }
+
+  /**
+   * Function to get the resources for a node. This function will look at the
+   * file {@link YarnConfiguration#NODE_RESOURCES_CONFIGURATION_FILE} to
+   * determine the node resources.
+   *
+   * @param conf configuration file
+   * @return a map to resource name to the ResourceInformation object. The map
+   * is guaranteed to have entries for memory and vcores
+   */
+  public static Map<String, ResourceInformation> getNodeResourceInformation(
+      Configuration conf) {
+    if (nodeLock == null) {
+      synchronized (ResourceUtils.class) {
+        if (nodeLock == null) {
+          synchronized (ResourceUtils.class) {
+            nodeLock = new Object();
+            Map<String, ResourceInformation> nodeResources =
+                initializeNodeResourceInformation(conf);
+            addManadtoryResources(nodeResources);
+            checkMandatatoryResources(nodeResources);
+            readOnlyNodeResources = Collections.unmodifiableMap(nodeResources);
+          }
+        }
+      }
+    }
+    return readOnlyNodeResources;
+  }
+
+  private static Map<String, ResourceInformation>
+  initializeNodeResourceInformation(Configuration conf) {
+    Map<String, ResourceInformation> nodeResources = new HashMap<>();
+    try {
+      addResourcesFileToConf(
+          YarnConfiguration.NODE_RESOURCES_CONFIGURATION_FILE, conf);
+      for (Map.Entry<String, String> entry : conf) {
+        String key = entry.getKey();
+        String value = entry.getValue();
+        if (key.startsWith(YarnConfiguration.NM_RESOURCES_PREFIX)) {
+          addResourceInformation(key, value, nodeResources);
+        }
+      }
+    } catch (FileNotFoundException fe) {
+      LOG.info("Couldn't find node resources file");
+    }
+    return nodeResources;
+  }
+
+  private static void addResourceInformation(String prop, String value,
+      Map<String, ResourceInformation> nodeResources) {
+    String[] parts = prop.split("\\.");
+    LOG.info("Found resource entry " + prop);
+    if (parts.length == 4) {
+      String resourceType = parts[3];
+      if (!nodeResources.containsKey(resourceType)) {
+        nodeResources
+            .put(resourceType, ResourceInformation.newInstance(resourceType));
+      }
+      String units = getUnits(value);
+      Long resourceValue =
+          Long.valueOf(value.substring(0, value.length() - units.length()));
+      nodeResources.get(resourceType).setValue(resourceValue);
+      nodeResources.get(resourceType).setUnits(units);
+      LOG.debug("Setting value for resource type " + resourceType + " to "
+              + resourceValue + " with units " + units);
+    }
+  }
+
+  @VisibleForTesting
+  synchronized public static void resetNodeResources() {
+    nodeLock = null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a0d5db2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java
index 4bb68c0..ab014dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java
@@ -21,9 +21,9 @@ package org.apache.hadoop.yarn.util.resource;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.util.SystemClock;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -245,4 +245,31 @@ public class TestResourceUtils {
       }
     }
   }
+
+  @Test
+  public void testGetResourceInformation() throws Exception {
+
+    Configuration conf = new YarnConfiguration();
+    Map<String, Resource> testRun = new HashMap<>();
+    // testRun.put("node-resources-1.xml", Resource.newInstance(1024, 1));
+    Resource test3Resources = Resource.newInstance(1024, 1);
+    test3Resources.setResourceInformation("resource1",
+        ResourceInformation.newInstance("resource1", "Gi", 5L));
+    test3Resources.setResourceInformation("resource2",
+        ResourceInformation.newInstance("resource2", "m", 2L));
+    testRun.put("node-resources-2.xml", test3Resources);
+
+    for (Map.Entry<String, Resource> entry : testRun.entrySet()) {
+      String resourceFile = entry.getKey();
+      ResourceUtils.resetNodeResources();
+      File dest;
+      File source =
+          new File(conf.getClassLoader().getResource(resourceFile).getFile());
+      dest = new File(source.getParent(), "node-resources.xml");
+      FileUtils.copyFile(source, dest);
+      Map<String, ResourceInformation> actual =
+          ResourceUtils.getNodeResourceInformation(conf);
+      Assert.assertEquals(entry.getValue().getResources(), actual);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a0d5db2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/node-resources-1.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/node-resources-1.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/node-resources-1.xml
new file mode 100644
index 0000000..f00573e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/node-resources-1.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License. See accompanying LICENSE file.
+-->
+
+<configuration>
+
+ <property>
+   <name>yarn.nodemanager.resource.memory-mb</name>
+   <value>1024</value>
+ </property>
+
+ <property>
+   <name>yarn.nodemanager.resource.vcores</name>
+   <value>1</value>
+ </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a0d5db2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/node-resources-2.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/node-resources-2.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/node-resources-2.xml
new file mode 100644
index 0000000..9d9b3dc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/node-resources-2.xml
@@ -0,0 +1,39 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License. See accompanying LICENSE file.
+-->
+
+<configuration>
+
+ <property>
+   <name>yarn.nodemanager.resource-type.memory-mb</name>
+   <value>1024Mi</value>
+ </property>
+
+ <property>
+   <name>yarn.nodemanager.resource-type.vcores</name>
+   <value>1</value>
+ </property>
+
+ <property>
+   <name>yarn.nodemanager.resource-type.resource1</name>
+   <value>5Gi</value>
+ </property>
+
+ <property>
+   <name>yarn.nodemanager.resource-type.resource2</name>
+   <value>2m</value>
+ </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a0d5db2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index f53e6c2..b98be76 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -172,18 +172,19 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
 
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
-    int memoryMb = NodeManagerHardwareUtils.getContainerMemoryMB(conf);
+    this.totalResource = NodeManagerHardwareUtils.getNodeResources(conf);
+    int memoryMb = totalResource.getMemory();
     float vMemToPMem =
         conf.getFloat(
             YarnConfiguration.NM_VMEM_PMEM_RATIO, 
             YarnConfiguration.DEFAULT_NM_VMEM_PMEM_RATIO); 
     int virtualMemoryMb = (int)Math.ceil(memoryMb * vMemToPMem);
     
-    int virtualCores = NodeManagerHardwareUtils.getVCores(conf);
+    int virtualCores = totalResource.getVirtualCores();
     LOG.info("Nodemanager resources: memory set to " + memoryMb + "MB.");
     LOG.info("Nodemanager resources: vcores set to " + virtualCores + ".");
+    LOG.info("Nodemanager resources: " + totalResource);
 
-    this.totalResource = Resource.newInstance(memoryMb, virtualCores);
     metrics.addResource(totalResource);
     this.tokenKeepAliveEnabled = isTokenKeepAliveEnabled(conf);
     this.tokenRemovalDelayMs =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a0d5db2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerHardwareUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerHardwareUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerHardwareUtils.java
index f3c95d3..a802917 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerHardwareUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/NodeManagerHardwareUtils.java
@@ -23,8 +23,14 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
+
+import java.util.Map;
 
 /**
  * Helper class to determine hardware related characteristics such as the
@@ -310,4 +316,50 @@ public class NodeManagerHardwareUtils {
     }
     return memoryMb;
   }
+
+  /**
+   * Get the resources for the node.
+   * @param configuration configuration file
+   * @return the resources for the node
+   */
+  public static Resource getNodeResources(Configuration configuration) {
+    Configuration conf = new Configuration(configuration);
+    String memory = ResourceInformation.MEMORY_MB.getName();
+    String vcores = ResourceInformation.VCORES.getName();
+
+    Resource ret = Resource.newInstance(0, 0);
+    Map<String, ResourceInformation> resourceInformation =
+        ResourceUtils.getNodeResourceInformation(conf);
+    for (Map.Entry<String, ResourceInformation> entry : resourceInformation
+        .entrySet()) {
+      ret.setResourceInformation(entry.getKey(), entry.getValue());
+      LOG.debug("Setting key " + entry.getKey() + " to " + entry.getValue());
+    }
+    if (resourceInformation.containsKey(memory)) {
+      Long value = resourceInformation.get(memory).getValue();
+      if (value > Integer.MAX_VALUE) {
+        throw new YarnRuntimeException("Value '" + value
+            + "' for resource memory is more than the maximum for an integer.");
+      }
+      ResourceInformation memResInfo = resourceInformation.get(memory);
+      if(memResInfo.getValue() == 0) {
+        ret.setMemory(getContainerMemoryMB(conf));
+        LOG.debug("Set memory to " + ret.getMemory());
+      }
+    }
+    if (resourceInformation.containsKey(vcores)) {
+      Long value = resourceInformation.get(vcores).getValue();
+      if (value > Integer.MAX_VALUE) {
+        throw new YarnRuntimeException("Value '" + value
+            + "' for resource vcores is more than the maximum for an integer.");
+      }
+      ResourceInformation vcoresResInfo = resourceInformation.get(vcores);
+      if(vcoresResInfo.getValue() == 0) {
+        ret.setVirtualCores(getVCores(conf));
+        LOG.debug("Set vcores to " + ret.getVirtualCores());
+      }
+    }
+    LOG.debug("Node resource information map is " + ret);
+    return ret;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a0d5db2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.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/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
index c55884e..2857ff8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
@@ -364,10 +364,11 @@ public class ResourceTrackerService extends AbstractService implements
     // Check if this node has minimum allocations
     if (capability.getMemorySize() < minAllocMb
         || capability.getVirtualCores() < minAllocVcores) {
-      String message =
-          "NodeManager from  " + host
-              + " doesn't satisfy minimum allocations, Sending SHUTDOWN"
-              + " signal to the NodeManager.";
+      String message = "NodeManager from  " + host
+          + " doesn't satisfy minimum allocations, Sending SHUTDOWN"
+          + " signal to the NodeManager. Node capabilities are " + capability
+          + "; minimums are " + minAllocMb + "mb and " + minAllocVcores
+          + " vcores";
       LOG.info(message);
       response.setDiagnosticsMessage(message);
       response.setNodeAction(NodeAction.SHUTDOWN);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[17/51] [abbrv] hadoop git commit: YARN-5191. Renamed the newly added “download=true” option for getting logs via NMWebServices and AHSWebServices to be a better "format" option. (Xuan Gong via vinodkv)

Posted by vv...@apache.org.
YARN-5191. Renamed the newly added \u201cdownload=true\u201d option for getting logs via NMWebServices and AHSWebServices to be a better "format" option. (Xuan Gong via vinodkv)


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

Branch: refs/heads/YARN-3926
Commit: 9378d9428f127eff7acd6c13544016cdbf2d65fb
Parents: 656c460
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Thu Jun 9 12:30:58 2016 -0700
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Thu Jun 9 12:30:58 2016 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/webapp/util/WebAppUtils.java    | 18 ++++++++++
 .../webapp/AHSWebServices.java                  | 36 ++++++++++++--------
 .../nodemanager/webapp/NMWebServices.java       | 33 +++++++++++-------
 .../nodemanager/webapp/TestNMWebServices.java   | 15 +++++++-
 4 files changed, 74 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9378d942/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
index faf4a77..3aa773a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
@@ -24,6 +24,7 @@ import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -400,4 +401,21 @@ public class WebAppUtils {
     }
     return aid;
   }
+
+  public static String getSupportedLogContentType(String format) {
+    if (format.equalsIgnoreCase("text")) {
+      return "text/plain";
+    } else if (format.equalsIgnoreCase("octet-stream")) {
+      return "application/octet-stream";
+    }
+    return null;
+  }
+
+  public static String getDefaultLogContentType() {
+    return "text/plain";
+  }
+
+  public static List<String> listSupportedLogContentType() {
+    return Arrays.asList("text", "octet-stream");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9378d942/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
index 59dbd44..692b172 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
@@ -66,6 +66,7 @@ import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import com.google.common.base.Joiner;
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
@@ -212,7 +213,7 @@ public class AHSWebServices extends WebServices {
       @Context HttpServletResponse res,
       @PathParam("containerid") String containerIdStr,
       @PathParam("filename") String filename,
-      @QueryParam("download") String download,
+      @QueryParam("format") String format,
       @QueryParam("size") String size) {
     init(res);
     ContainerId containerId;
@@ -223,9 +224,6 @@ public class AHSWebServices extends WebServices {
           "Invalid ContainerId: " + containerIdStr);
     }
 
-    boolean downloadFile = parseBooleanParam(download);
-
-
     final long length = parseLongParam(size);
 
     ApplicationId appId = containerId.getApplicationAttemptId()
@@ -236,7 +234,7 @@ public class AHSWebServices extends WebServices {
     } catch (Exception ex) {
       // directly find logs from HDFS.
       return sendStreamOutputResponse(appId, null, null, containerIdStr,
-          filename, downloadFile, length);
+          filename, format, length);
     }
     String appOwner = appInfo.getUser();
 
@@ -250,7 +248,7 @@ public class AHSWebServices extends WebServices {
       if (isFinishedState(appInfo.getAppState())) {
         // directly find logs from HDFS.
         return sendStreamOutputResponse(appId, appOwner, null, containerIdStr,
-            filename, downloadFile, length);
+            filename, format, length);
       }
       return createBadResponse(Status.INTERNAL_SERVER_ERROR,
           "Can not get ContainerInfo for the container: " + containerId);
@@ -270,7 +268,7 @@ public class AHSWebServices extends WebServices {
       return response.build();
     } else if (isFinishedState(appInfo.getAppState())) {
       return sendStreamOutputResponse(appId, appOwner, nodeId,
-          containerIdStr, filename, downloadFile, length);
+          containerIdStr, filename, format, length);
     } else {
       return createBadResponse(Status.NOT_FOUND,
           "The application is not at Running or Finished State.");
@@ -293,13 +291,19 @@ public class AHSWebServices extends WebServices {
     return response;
   }
 
-  private boolean parseBooleanParam(String param) {
-    return ("true").equalsIgnoreCase(param);
-  }
-
   private Response sendStreamOutputResponse(ApplicationId appId,
       String appOwner, String nodeId, String containerIdStr,
-      String fileName, boolean downloadFile, long bytes) {
+      String fileName, String format, long bytes) {
+    String contentType = WebAppUtils.getDefaultLogContentType();
+    if (format != null && !format.isEmpty()) {
+      contentType = WebAppUtils.getSupportedLogContentType(format);
+      if (contentType == null) {
+        String errorMessage = "The valid values for the parameter : format "
+            + "are " + WebAppUtils.listSupportedLogContentType();
+        return Response.status(Status.BAD_REQUEST).entity(errorMessage)
+            .build();
+      }
+    }
     StreamingOutput stream = null;
     try {
       stream = getStreamingOutput(appId, appOwner, nodeId,
@@ -313,9 +317,11 @@ public class AHSWebServices extends WebServices {
           "Can not get log for container: " + containerIdStr);
     }
     ResponseBuilder response = Response.ok(stream);
-    if (downloadFile) {
-      response.header("Content-Type", "application/octet-stream");
-    }
+    response.header("Content-Type", contentType);
+    // Sending the X-Content-Type-Options response header with the value
+    // nosniff will prevent Internet Explorer from MIME-sniffing a response
+    // away from the declared content-type.
+    response.header("X-Content-Type-Options", "nosniff");
     return response.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9378d942/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
index 943f3cc..efc0e7e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
@@ -206,6 +206,10 @@ public class NMWebServices {
    *    The container ID
    * @param filename
    *    The name of the log file
+   * @param format
+   *    The content type
+   * @param size
+   *    the size of the log file
    * @return
    *    The contents of the container's log file
    */
@@ -216,7 +220,7 @@ public class NMWebServices {
   @Unstable
   public Response getLogs(@PathParam("containerid") String containerIdStr,
       @PathParam("filename") String filename,
-      @QueryParam("download") String download,
+      @QueryParam("format") String format,
       @QueryParam("size") String size) {
     ContainerId containerId;
     try {
@@ -234,8 +238,18 @@ public class NMWebServices {
     } catch (YarnException ex) {
       return Response.serverError().entity(ex.getMessage()).build();
     }
-    boolean downloadFile = parseBooleanParam(download);
     final long bytes = parseLongParam(size);
+    String contentType = WebAppUtils.getDefaultLogContentType();
+    if (format != null && !format.isEmpty()) {
+      contentType = WebAppUtils.getSupportedLogContentType(format);
+      if (contentType == null) {
+        String errorMessage = "The valid values for the parameter : format "
+            + "are " + WebAppUtils.listSupportedLogContentType();
+        return Response.status(Status.BAD_REQUEST).entity(errorMessage)
+            .build();
+      }
+    }
+
     try {
       final FileInputStream fis = ContainerLogsUtils.openLogFileForRead(
           containerIdStr, logFile, nmContext);
@@ -288,22 +302,17 @@ public class NMWebServices {
         }
       };
       ResponseBuilder resp = Response.ok(stream);
-      if (downloadFile) {
-        resp.header("Content-Type", "application/octet-stream");
-      }
+      resp.header("Content-Type", contentType);
+      // Sending the X-Content-Type-Options response header with the value
+      // nosniff will prevent Internet Explorer from MIME-sniffing a response
+      // away from the declared content-type.
+      resp.header("X-Content-Type-Options", "nosniff");
       return resp.build();
     } catch (IOException ex) {
       return Response.serverError().entity(ex.getMessage()).build();
     }
   }
 
-  private boolean parseBooleanParam(String param) {
-    if (param != null) {
-      return ("true").equalsIgnoreCase(param);
-    }
-    return false;
-  }
-
   private long parseLongParam(String bytes) {
     if (bytes == null || bytes.isEmpty()) {
       return Long.MAX_VALUE;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9378d942/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.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/webapp/TestNMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
index 4e2feee..a4305da 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.AfterClass;
@@ -389,18 +390,30 @@ public class TestNMWebServices extends JerseyTestBase {
         .queryParam("size", "-10000")
         .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
     responseText = response.getEntity(String.class);
+    assertEquals("text/plain", response.getType().toString());
     assertEquals(fullTextSize, responseText.getBytes().length);
     assertEquals(logMessage, responseText);
 
     // ask and download it
     response = r.path("ws").path("v1").path("node").path("containerlogs")
-        .path(containerIdStr).path(filename).queryParam("download", "true")
+        .path(containerIdStr).path(filename)
+        .queryParam("format", "octet-stream")
         .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
     responseText = response.getEntity(String.class);
     assertEquals(logMessage, responseText);
     assertEquals(200, response.getStatus());
     assertEquals("application/octet-stream", response.getType().toString());
 
+    // specify a invalid format value
+    response = r.path("ws").path("v1").path("node").path("containerlogs")
+        .path(containerIdStr).path(filename)
+        .queryParam("format", "123")
+        .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
+    responseText = response.getEntity(String.class);
+    assertEquals("The valid values for the parameter : format are "
+        + WebAppUtils.listSupportedLogContentType(), responseText);
+    assertEquals(400, response.getStatus());
+
     // ask for file that doesn't exist
     response = r.path("ws").path("v1").path("node")
         .path("containerlogs").path(containerIdStr).path("uhhh")


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[26/51] [abbrv] hadoop git commit: YARN-3426. Add jdiff support to YARN. (vinodkv via wangda)

Posted by vv...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/03fc6b1b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Null.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Null.java b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Null.java
new file mode 100644
index 0000000..7b00145
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Null.java
@@ -0,0 +1,20 @@
+/**
+ * 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.
+ */
+public class Null {
+  public Null() { }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03fc6b1b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
index 187dbbb..41aef33 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
@@ -30,6 +30,8 @@
   <properties>
     <!-- Needed for generating FindBugs warnings using parent pom -->
     <yarn.basedir>${project.parent.basedir}</yarn.basedir>
+    <should.run.jdiff>true</should.run.jdiff>
+    <dev-support.relative.dir>../dev-support</dev-support.relative.dir>
   </properties>
 
   <dependencies>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03fc6b1b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
index d6ff6af..df15c7c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
@@ -27,6 +27,8 @@
   <properties>
     <!-- Needed for generating FindBugs warnings using parent pom -->
     <yarn.basedir>${project.parent.basedir}</yarn.basedir>
+    <should.run.jdiff>true</should.run.jdiff>
+    <dev-support.relative.dir>../dev-support</dev-support.relative.dir>
   </properties>
 
   <dependencies>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03fc6b1b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
index f13d6ec..17fc6e2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
@@ -30,6 +30,8 @@
   <properties>
     <!-- Needed for generating FindBugs warnings using parent pom -->
     <yarn.basedir>${project.parent.basedir}</yarn.basedir>
+    <should.run.jdiff>true</should.run.jdiff>
+    <dev-support.relative.dir>../dev-support</dev-support.relative.dir>
   </properties>
 
   <dependencies>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03fc6b1b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
index ad9f977..f792ccd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
@@ -30,6 +30,8 @@
   <properties>
     <!-- Needed for generating FindBugs warnings using parent pom -->
     <yarn.basedir>${project.parent.parent.basedir}</yarn.basedir>
+    <should.run.jdiff>true</should.run.jdiff>
+    <dev-support.relative.dir>../../dev-support</dev-support.relative.dir>
   </properties>
 
   <dependencies>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03fc6b1b/hadoop-yarn-project/hadoop-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/pom.xml b/hadoop-yarn-project/hadoop-yarn/pom.xml
index 0f79226..3e31ec0 100644
--- a/hadoop-yarn-project/hadoop-yarn/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/pom.xml
@@ -29,6 +29,9 @@
     <test.logs>true</test.logs>
     <test.timeout>600000</test.timeout>
     <yarn.basedir>${basedir}</yarn.basedir>
+    <!-- Used by jdiff -->
+    <!-- Antrun cannot resolve yarn.basedir, so we need to setup something else -->
+    <dev-support.relative.dir>dev-support</dev-support.relative.dir>
     <hadoop.common.build.dir>${basedir}/../../../hadoop-common-project/hadoop-common/target</hadoop.common.build.dir>
   </properties>
 
@@ -53,6 +56,7 @@
           <excludes>
             <exclude>conf/slaves</exclude>
             <exclude>conf/container-executor.cfg</exclude>
+            <exclude>dev-support/jdiff/**</exclude>
           </excludes>
         </configuration>
       </plugin>
@@ -98,6 +102,131 @@
         </plugins>
       </build>
     </profile>
+
+    <profile>
+      <id>docs</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <properties>
+        <jdiff.stable.api>2.7.2</jdiff.stable.api>
+        <jdiff.stability>-unstable</jdiff.stability>
+        <jdiff.compatibility></jdiff.compatibility>
+        <jdiff.javadoc.maxmemory>512m</jdiff.javadoc.maxmemory>
+      </properties>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-javadoc-plugin</artifactId>
+            <executions>
+              <execution>
+                <goals>
+                  <goal>javadoc</goal>
+                </goals>
+                <phase>prepare-package</phase>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-dependency-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>site</id>
+                <phase>prepare-package</phase>
+                <goals>
+                  <goal>copy</goal>
+                </goals>
+                <configuration>
+                  <artifactItems>
+                    <artifactItem>
+                      <groupId>jdiff</groupId>
+                      <artifactId>jdiff</artifactId>
+                      <version>${jdiff.version}</version>
+                      <overWrite>false</overWrite>
+                      <outputDirectory>${project.build.directory}</outputDirectory>
+                      <destFileName>jdiff.jar</destFileName>
+                    </artifactItem>
+                    <artifactItem>
+                      <groupId>org.apache.hadoop</groupId>
+                      <artifactId>hadoop-annotations</artifactId>
+                      <version>${project.version}</version>
+                      <overWrite>false</overWrite>
+                      <outputDirectory>${project.build.directory}</outputDirectory>
+                      <destFileName>hadoop-annotations.jar</destFileName>
+                    </artifactItem>
+                    <artifactItem>
+                      <groupId>xerces</groupId>
+                      <artifactId>xercesImpl</artifactId>
+                      <version>${xerces.version.jdiff}</version>
+                      <overWrite>false</overWrite>
+                      <outputDirectory>${project.build.directory}</outputDirectory>
+                      <destFileName>xerces.jar</destFileName>
+                    </artifactItem>
+                  </artifactItems>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>site</id>
+                <phase>prepare-package</phase>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+                <configuration>
+                  <target if="should.run.jdiff">
+
+                    <!-- Jdiff -->
+                    <mkdir dir="${project.build.directory}/site/jdiff/xml"/>
+
+                    <javadoc maxmemory="${jdiff.javadoc.maxmemory}" verbose="yes">
+                      <doclet name="org.apache.hadoop.classification.tools.ExcludePrivateAnnotationsJDiffDoclet"
+                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar">
+                        <param name="-apidir" value="${project.build.directory}/site/jdiff/xml"/>
+                        <param name="-apiname" value="${project.name} ${project.version}"/>
+                        <param name="${jdiff.stability}"/>
+                      </doclet>
+                      <packageset dir="${basedir}/src/main/java"/>
+                      <classpath>
+                        <path refid="maven.compile.classpath"/>
+                      </classpath>
+                    </javadoc>
+                    <javadoc sourcepath="${basedir}/src/main/java"
+                      destdir="${project.build.directory}/site/jdiff/xml"
+                      sourceFiles="${dev-support.relative.dir}/jdiff/Null.java"
+                             maxmemory="${jdiff.javadoc.maxmemory}">
+                      <doclet name="org.apache.hadoop.classification.tools.ExcludePrivateAnnotationsJDiffDoclet"
+                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar:${project.build.directory}/xerces.jar">
+                        <param name="-oldapi" value="${project.name} ${jdiff.stable.api}"/>
+                        <param name="-newapi" value="${project.name} ${project.version}"/>
+                        <param name="-oldapidir" value="${basedir}/${dev-support.relative.dir}/jdiff"/>
+                        <param name="-newapidir" value="${project.build.directory}/site/jdiff/xml"/>
+                        <param name="-javadocold"
+                               value="http://hadoop.apache.org/docs/r${jdiff.stable.api}/api/"/>
+                        <param name="-javadocnew" value="${project.build.directory}/site/apidocs/"/>
+                        <param name="-stats"/>
+                        <param name="${jdiff.stability}"/>
+                        <!--param name="${jdiff.compatibility}"/-->
+                      </doclet>
+                      <classpath>
+                        <path refid="maven.compile.classpath"/>
+                      </classpath>
+                    </javadoc>
+
+                  </target>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
   </profiles>
 
   <modules>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[20/51] [abbrv] hadoop git commit: HADOOP-12893. Verify LICENSE.txt and NOTICE.txt. Contributed by Xiao Chen, Akira Ajisaka, and Andrew Wang.

Posted by vv...@apache.org.
HADOOP-12893. Verify LICENSE.txt and NOTICE.txt. Contributed by Xiao Chen, Akira Ajisaka, and Andrew Wang.


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

Branch: refs/heads/YARN-3926
Commit: e383b732c54c542482b0b836e2d2c46eb49b4e2d
Parents: 58c3486
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Jun 9 13:54:14 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Jun 9 13:54:14 2016 -0700

----------------------------------------------------------------------
 LICENSE.txt                 | 1017 +++++++++++++++++++++++++++++++++++++-
 NOTICE.txt                  |  266 ++++++++++
 hadoop-build-tools/pom.xml  |   41 ++
 hadoop-project-dist/pom.xml |    2 +
 hadoop-project/pom.xml      |   19 +-
 pom.xml                     |   26 +
 6 files changed, 1367 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e383b732/LICENSE.txt
----------------------------------------------------------------------
diff --git a/LICENSE.txt b/LICENSE.txt
index 929e2a8..44880df 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -320,7 +320,9 @@ THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
 (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
 OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
 
-For com.google.re2j.* classes:
+The binary distribution of this product bundles these dependencies under the
+following license:
+re2j 1.0
 ---------------------------------------------------------------------
 This is a work derived from Russ Cox's RE2 in Go, whose license
 http://golang.org/LICENSE is as follows:
@@ -548,12 +550,14 @@ hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2
 hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/bootstrap.min.js
 hadoop-tools/hadoop-sls/src/main/html/css/bootstrap.min.css
 hadoop-tools/hadoop-sls/src/main/html/css/bootstrap-responsive.min.css
+And the binary distribution of this product bundles these dependencies under the
+following license:
+Mockito 1.8.5
+SLF4J 1.7.10
 --------------------------------------------------------------------------------
 
 The MIT License (MIT)
 
-Copyright (c) 2011-2016 Twitter, Inc.
-
 Permission is hereby granted, free of charge, to any person obtaining a copy
 of this software and associated documentation files (the "Software"), to deal
 in the Software without restriction, including without limitation the rights
@@ -648,3 +652,1010 @@ hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3.v3.js
 
 D3 is available under a 3-clause BSD license. For details, see:
 hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3-LICENSE
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+HSQLDB Database 2.0.0
+--------------------------------------------------------------------------------
+"COPYRIGHTS AND LICENSES (based on BSD License)
+
+For work developed by the HSQL Development Group:
+
+Copyright (c) 2001-2016, The HSQL Development Group
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice, this
+list of conditions and the following disclaimer.
+
+Redistributions in binary form must reproduce the above copyright notice,
+this list of conditions and the following disclaimer in the documentation
+and/or other materials provided with the distribution.
+
+Neither the name of the HSQL Development Group nor the names of its
+contributors may be used to endorse or promote products derived from this
+software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS""
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ARE DISCLAIMED. IN NO EVENT SHALL HSQL DEVELOPMENT GROUP, HSQLDB.ORG,
+OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+For work originally developed by the Hypersonic SQL Group:
+
+Copyright (c) 1995-2000 by the Hypersonic SQL Group.
+All rights reserved.
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice, this
+list of conditions and the following disclaimer.
+
+Redistributions in binary form must reproduce the above copyright notice,
+this list of conditions and the following disclaimer in the documentation
+and/or other materials provided with the distribution.
+
+Neither the name of the Hypersonic SQL Group nor the names of its
+contributors may be used to endorse or promote products derived from this
+software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS""
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ARE DISCLAIMED. IN NO EVENT SHALL THE HYPERSONIC SQL GROUP,
+OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+This software consists of voluntary contributions made by many individuals on behalf of the
+Hypersonic SQL Group."
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+servlet-api 2.5
+jsp-api 2.1
+Streaming API for XML 1.0
+--------------------------------------------------------------------------------
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0
+1.�Definitions.�
+
+1.1.�Contributor�means each individual or entity
+that creates or contributes to the creation of
+Modifications.�
+
+1.2.�Contributor Version�means the combination of the
+Original Software, prior Modifications used by a Contributor (if any), and the
+Modifications made by that particular Contributor.�
+
+1.3.�Covered
+Software�means (a) the Original Software, or (b) Modifications, or (c) the
+combination of files containing Original Software with files containing
+Modifications, in each case including portions
+thereof.�
+
+1.4.�Executable�means the Covered Software in any form other
+than Source Code.�
+
+1.5.�Initial Developer�means the individual or entity
+that first makes Original Software available under this
+License.�
+
+1.6.�Larger Work�means a work which combines Covered Software or
+portions thereof with code not governed by the terms of this
+License.�
+
+1.7.�License�means this document.�
+
+1.8.�Licensable�means
+having the right to grant, to the maximum extent possible, whether at the time
+of the initial grant or subsequently acquired, any and all of the rights
+conveyed herein.�
+
+1.9.�Modifications�means the Source Code and Executable
+form of any of the following:
+A. Any file that results from an addition to,
+deletion from or modification of the contents of a file containing Original
+Software or previous Modifications;
+B. Any new file that contains any part of the Original Software
+or previous Modification; or
+C. Any new file that is contributed or otherwise made available
+under the terms of this License.�
+
+1.10.�Original Software�means the Source Code and Executable form of
+computer software code that is originally released under this License.�
+
+1.11.�Patent Claims�means any patent claim(s), now owned or
+hereafter acquired, including without limitation, method, process, and apparatus
+claims, in any patent Licensable by grantor.�
+
+1.12.�Source Code�means (a) the common form of computer software code in which
+modifications are made and (b) associated documentation included in or
+with such code.�
+
+1.13.�You (or Your)�means an individual or a legal entity exercising rights
+under, and complying with all of the terms of, this License. For legal entities,
+You includes any entity which controls, is controlled by, or is under common control
+with You. For purposes of this definition, control means (a)�the power, direct
+or indirect, to cause the direction or management of such entity, whether by
+contract or otherwise, or (b)�ownership of more than fifty percent (50%) of the
+outstanding shares or beneficial ownership of such entity.�
+
+2. License Grants.
+
+2.1. The Initial Developer Grant. Conditioned upon Your compliance
+with Section 3.1 below and subject to third party intellectual property claims,
+the Initial Developer hereby grants You a world-wide, royalty-free,
+non-exclusive license:�
+
+(a) under intellectual property rights (other than
+patent or trademark) Licensable by Initial Developer, to use, reproduce, modify,
+display, perform, sublicense and distribute the Original Software (or portions
+thereof), with or without Modifications, and/or as part of a Larger Work;
+and�
+
+(b) under Patent Claims infringed by the making, using or selling of
+Original Software, to make, have made, use, practice, sell, and offer for sale,
+and/or otherwise dispose of the Original Software (or portions
+thereof);
+
+(c) The licenses granted in Sections�2.1(a) and (b) are
+effective on the date Initial Developer first distributes or otherwise makes the
+Original Software available to a third party under the terms of this
+License;
+
+(d) Notwithstanding Section�2.1(b) above, no patent license is
+granted: (1)�for code that You delete from the Original Software, or (2)�for
+infringements caused by: (i)�the modification of the Original Software, or
+(ii)�the combination of the Original Software with other software or
+devices.�
+
+2.2. Contributor Grant. Conditioned upon Your compliance with
+Section 3.1 below and subject to third party intellectual property claims, each
+Contributor hereby grants You a world-wide, royalty-free, non-exclusive
+license:�
+
+(a) under intellectual property rights (other than patent or
+trademark) Licensable by Contributor to use, reproduce, modify, display,
+perform, sublicense and distribute the Modifications created by such Contributor
+(or portions thereof), either on an unmodified basis, with other Modifications,
+as Covered Software and/or as part of a Larger Work; and�
+
+(b) under Patent
+Claims infringed by the making, using, or selling of Modifications made by that
+Contributor either alone and/or in combination with its Contributor Version (or
+portions of such combination), to make, use, sell, offer for sale, have made,
+and/or otherwise dispose of: (1)�Modifications made by that Contributor (or
+portions thereof); and (2)�the combination of Modifications made by that
+Contributor with its Contributor Version (or portions of such
+combination).�
+
+(c) The licenses granted in Sections�2.2(a) and 2.2(b) are
+effective on the date Contributor first distributes or otherwise makes the
+Modifications available to a third party.
+
+(d) Notwithstanding Section�2.2(b)
+above, no patent license is granted: (1)�for any code that Contributor has
+deleted from the Contributor Version; (2)�for infringements caused by:
+(i)�third party modifications of Contributor Version, or (ii)�the combination
+of Modifications made by that Contributor with other software (except as part of
+the Contributor Version) or other devices; or (3)�under Patent Claims infringed
+by Covered Software in the absence of Modifications made by that
+Contributor.�
+
+3. Distribution Obligations.�
+
+3.1. Availability of Source
+Code. Any Covered Software that You distribute or otherwise make available in
+Executable form must also be made available in Source Code form and that Source
+Code form must be distributed only under the terms of this License. You must
+include a copy of this License with every copy of the Source Code form of the
+Covered Software You distribute or otherwise make available. You must inform
+recipients of any such Covered Software in Executable form as to how they can
+obtain such Covered Software in Source Code form in a reasonable manner on or
+through a medium customarily used for software exchange.�
+
+3.2.
+Modifications. The Modifications that You create or to which You contribute are
+governed by the terms of this License. You represent that You believe Your
+Modifications are Your original creation(s) and/or You have sufficient rights to
+grant the rights conveyed by this License.�
+
+3.3. Required Notices. You must
+include a notice in each of Your Modifications that identifies You as the
+Contributor of the Modification. You may not remove or alter any copyright,
+patent or trademark notices contained within the Covered Software, or any
+notices of licensing or any descriptive text giving attribution to any
+Contributor or the Initial Developer.�
+
+3.4. Application of Additional Terms.
+You may not offer or impose any terms on any Covered Software in Source Code
+form that alters or restricts the applicable version of this License or the
+recipients rights hereunder. You may choose to offer, and to charge a fee for,
+warranty, support, indemnity or liability obligations to one or more recipients
+of Covered Software. However, you may do so only on Your own behalf, and not on
+behalf of the Initial Developer or any Contributor. You must make it absolutely
+clear that any such warranty, support, indemnity or liability obligation is
+offered by You alone, and You hereby agree to indemnify the Initial Developer
+and every Contributor for any liability incurred by the Initial Developer or
+such Contributor as a result of warranty, support, indemnity or liability terms
+You offer.
+
+3.5. Distribution of Executable Versions. You may distribute the
+Executable form of the Covered Software under the terms of this License or under
+the terms of a license of Your choice, which may contain terms different from
+this License, provided that You are in compliance with the terms of this License
+and that the license for the Executable form does not attempt to limit or alter
+the recipients rights in the Source Code form from the rights set forth in this
+License. If You distribute the Covered Software in Executable form under a
+different license, You must make it absolutely clear that any terms which differ
+from this License are offered by You alone, not by the Initial Developer or
+Contributor. You hereby agree to indemnify the Initial Developer and every
+Contributor for any liability incurred by the Initial Developer or such
+Contributor as a result of any such terms You offer.�
+
+3.6. Larger Works. You
+may create a Larger Work by combining Covered Software with other code not
+governed by the terms of this License and distribute the Larger Work as a single
+product. In such a case, You must make sure the requirements of this License are
+fulfilled for the Covered Software.�
+
+4. Versions of the License.�
+
+4.1.
+New Versions. Sun Microsystems, Inc. is the initial license steward and may
+publish revised and/or new versions of this License from time to time. Each
+version will be given a distinguishing version number. Except as provided in
+Section 4.3, no one other than the license steward has the right to modify this
+License.�
+
+4.2. Effect of New Versions. You may always continue to use,
+distribute or otherwise make the Covered Software available under the terms of
+the version of the License under which You originally received the Covered
+Software. If the Initial Developer includes a notice in the Original Software
+prohibiting it from being distributed or otherwise made available under any
+subsequent version of the License, You must distribute and make the Covered
+Software available under the terms of the version of the License under which You
+originally received the Covered Software. Otherwise, You may also choose to use,
+distribute or otherwise make the Covered Software available under the terms of
+any subsequent version of the License published by the license
+steward.�
+
+4.3. Modified Versions. When You are an Initial Developer and You
+want to create a new license for Your Original Software, You may create and use
+a modified version of this License if You: (a)�rename the license and remove
+any references to the name of the license steward (except to note that the
+license differs from this License); and (b)�otherwise make it clear that the
+license contains terms which differ from this License.�
+
+5. DISCLAIMER OF WARRANTY.
+
+COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN AS IS BASIS,
+WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT
+LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS,
+MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS
+TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY
+COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER
+OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR
+CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS
+LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER
+THIS DISCLAIMER.�
+
+6. TERMINATION.�
+
+6.1. This License and the rights
+granted hereunder will terminate automatically if You fail to comply with terms
+herein and fail to cure such breach within 30 days of becoming aware of the
+breach. Provisions which, by their nature, must remain in effect beyond the
+termination of this License shall survive.�
+
+6.2. If You assert a patent
+infringement claim (excluding declaratory judgment actions) against Initial
+Developer or a Contributor (the Initial Developer or Contributor against whom
+You assert such claim is referred to as Participant) alleging that the
+Participant Software (meaning the Contributor Version where the Participant is a
+Contributor or the Original Software where the Participant is the Initial
+Developer) directly or indirectly infringes any patent, then any and all rights
+granted directly or indirectly to You by such Participant, the Initial Developer
+(if the Initial Developer is not the Participant) and all Contributors under
+Sections�2.1 and/or 2.2 of this License shall, upon 60 days notice from
+Participant terminate prospectively and automatically at the expiration of such
+60 day notice period, unless if within such 60 day period You withdraw Your
+claim with respect to the Participant Software against such Participant either
+unilaterally or pursuant to a written agreement with Participant.�
+
+6.3. In
+the event of termination under Sections�6.1 or 6.2 above, all end user licenses
+that have been validly granted by You or any distributor hereunder prior to
+termination (excluding licenses granted to You by any distributor) shall survive
+termination.�
+
+7. LIMITATION OF LIABILITY.
+UNDER NO CIRCUMSTANCES AND UNDER
+NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE,
+SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF
+COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY
+PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY
+CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF
+GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER
+COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE
+POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO
+LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTYS NEGLIGENCE TO
+THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT
+ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO
+THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU.�
+
+8. U.S. GOVERNMENT END USERS.
+
+The Covered Software is a commercial item, as that term is defined in
+48�C.F.R.�2.101 (Oct. 1995), consisting of commercial computer software (as
+that term is defined at 48 C.F.R. �252.227-7014(a)(1)) and commercial computer
+software documentation as such terms are used in 48�C.F.R.�12.212 (Sept.
+1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through
+227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software
+with only those rights set forth herein. This U.S. Government Rights clause is
+in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision
+that addresses Government rights in computer software under this
+License.�
+
+9. MISCELLANEOUS.
+This License represents the complete agreement
+concerning subject matter hereof. If any provision of this License is held to be
+unenforceable, such provision shall be reformed only to the extent necessary to
+make it enforceable. This License shall be governed by the law of the
+jurisdiction specified in a notice contained within the Original Software
+(except to the extent applicable law, if any, provides otherwise), excluding
+such jurisdictions conflict-of-law provisions. Any litigation relating to this
+License shall be subject to the jurisdiction of the courts located in the
+jurisdiction and venue specified in a notice contained within the Original
+Software, with the losing party responsible for costs, including, without
+limitation, court costs and reasonable attorneys fees and expenses. The
+application of the United Nations Convention on Contracts for the International
+Sale of Goods is expressly excluded. Any law or regulation which provides that
+the language of a contract shall be construed against the drafter shall not
+apply to this License. You agree that You alone are responsible for compliance
+with the United States export administration regulations (and the export control
+laws and regulation of any other countries) when You use, distribute or
+otherwise make available any Covered Software.�
+
+10. RESPONSIBILITY FOR CLAIMS.
+As between Initial Developer and the Contributors, each party is
+responsible for claims and damages arising, directly or indirectly, out of its
+utilization of rights under this License and You agree to work with Initial
+Developer and Contributors to distribute such responsibility on an equitable
+basis. Nothing herein is intended or shall be deemed to constitute any admission
+of liability.�
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+Jersey 1.9
+JAXB API bundle for GlassFish V3 2.2.2
+JAXB RI 2.2.3
+--------------------------------------------------------------------------------
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)Version 1.1
+
+1. Definitions.
+
+1.1. \u201cContributor\u201d means each individual or entity that creates or
+contributes to the creation of Modifications.
+1.2. \u201cContributor Version\u201d means the combination of the Original Software,
+prior Modifications used by a Contributor (if any), and the Modifications made
+by that particular Contributor.
+1.3. \u201cCovered Software\u201d means (a) the Original Software, or (b)
+Modifications, or (c) the combination of files containing Original Software with
+files containing Modifications, in each case including portions thereof.
+1.4. \u201cExecutable\u201d means the Covered Software in any form other than Source
+Code.
+1.5. \u201cInitial Developer\u201d means the individual or entity that first makes
+Original Software available under this License.
+1.6. \u201cLarger Work\u201d means a work which combines Covered Software or portions
+thereof with code not governed by the terms of this License.
+1.7. \u201cLicense\u201d means this document.
+1.8. \u201cLicensable\u201d means having the right to grant, to the maximum extent
+possible, whether at the time of the initial grant or subsequently acquired, any
+and all of the rights conveyed herein.
+1.9. \u201cModifications\u201d means the Source Code and Executable form of any of the
+following:
+A. Any file that results from an addition to, deletion from or modification of
+the contents of a file containing Original Software or previous Modifications;
+B. Any new file that contains any part of the Original Software or previous
+Modification; or
+C. Any new file that is contributed or otherwise made available under the terms
+of this License.
+1.10. \u201cOriginal Software\u201d means the Source Code and Executable form of
+computer software code that is originally released under this License.
+1.11. \u201cPatent Claims\u201d means any patent claim(s), now owned or hereafter
+acquired, including without limitation, method, process, and apparatus claims,
+in any patent Licensable by grantor.
+1.12. \u201cSource Code\u201d means (a) the common form of computer software code in
+which modifications are made and (b) associated documentation included in or
+with such code.
+1.13. \u201cYou\u201d (or \u201cYour\u201d) means an individual or a legal entity exercising
+rights under, and complying with all of the terms of, this License. For legal
+entities, \u201cYou\u201d includes any entity which controls, is controlled by, or is
+under common control with You. For purposes of this definition, \u201ccontrol\u201d
+means (a) the power, direct or indirect, to cause the direction or management of
+such entity, whether by contract or otherwise, or (b) ownership of more than
+fifty percent (50%) of the outstanding shares or beneficial ownership of such
+entity.
+
+2. License Grants.
+
+2.1. The Initial Developer Grant.
+
+Conditioned upon Your compliance with Section 3.1 below and subject to
+third party intellectual property claims, the Initial Developer hereby grants
+You a world-wide, royalty-free, non-exclusive license:
+(a) under intellectual
+property rights (other than patent or trademark) Licensable by Initial
+Developer, to use, reproduce, modify, display, perform, sublicense and
+distribute the Original Software (or portions thereof), with or without
+Modifications, and/or as part of a Larger Work; and
+(b) under Patent Claims
+infringed by the making, using or selling of Original Software, to make, have
+made, use, practice, sell, and offer for sale, and/or otherwise dispose of the
+Original Software (or portions thereof).
+(c) The licenses granted in Sections
+2.1(a) and (b) are effective on the date Initial Developer first distributes or
+otherwise makes the Original Software available to a third party under the terms
+of this License.
+(d) Notwithstanding Section 2.1(b) above, no patent license is
+granted: (1) for code that You delete from the Original Software, or (2) for
+infringements caused by: (i) the modification of the Original Software, or (ii)
+the combination of the Original Software with other software or devices.
+
+2.2. Contributor Grant.
+
+Conditioned upon Your compliance with Section 3.1 below and
+subject to third party intellectual property claims, each Contributor hereby
+grants You a world-wide, royalty-free, non-exclusive license:
+(a) under
+intellectual property rights (other than patent or trademark) Licensable by
+Contributor to use, reproduce, modify, display, perform, sublicense and
+distribute the Modifications created by such Contributor (or portions thereof),
+either on an unmodified basis, with other Modifications, as Covered Software
+and/or as part of a Larger Work; and
+(b) under Patent Claims infringed by the
+making, using, or selling of Modifications made by that Contributor either alone
+and/or in combination with its Contributor Version (or portions of such
+combination), to make, use, sell, offer for sale, have made, and/or otherwise
+dispose of: (1) Modifications made by that Contributor (or portions thereof);
+and (2) the combination of Modifications made by that Contributor with its
+Contributor Version (or portions of such combination).
+(c) The licenses granted
+in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first
+distributes or otherwise makes the Modifications available to a third
+party.
+(d) Notwithstanding Section 2.2(b) above, no patent license is granted:
+(1) for any code that Contributor has deleted from the Contributor Version; (2)
+for infringements caused by: (i) third party modifications of Contributor
+Version, or (ii) the combination of Modifications made by that Contributor with
+other software (except as part of the Contributor Version) or other devices; or
+(3) under Patent Claims infringed by Covered Software in the absence of
+Modifications made by that Contributor.
+
+3. Distribution Obligations.
+
+3.1. Availability of Source Code.
+Any Covered Software that You distribute or
+otherwise make available in Executable form must also be made available in
+Source Code form and that Source Code form must be distributed only under the
+terms of this License. You must include a copy of this License with every copy
+of the Source Code form of the Covered Software You distribute or otherwise make
+available. You must inform recipients of any such Covered Software in Executable
+form as to how they can obtain such Covered Software in Source Code form in a
+reasonable manner on or through a medium customarily used for software
+exchange.
+3.2. Modifications.
+The Modifications that You create or to which
+You contribute are governed by the terms of this License. You represent that You
+believe Your Modifications are Your original creation(s) and/or You have
+sufficient rights to grant the rights conveyed by this License.
+3.3. Required Notices.
+You must include a notice in each of Your Modifications that
+identifies You as the Contributor of the Modification. You may not remove or
+alter any copyright, patent or trademark notices contained within the Covered
+Software, or any notices of licensing or any descriptive text giving attribution
+to any Contributor or the Initial Developer.
+3.4. Application of Additional Terms.
+You may not offer or impose any terms on any Covered Software in Source
+Code form that alters or restricts the applicable version of this License or the
+recipients' rights hereunder. You may choose to offer, and to charge a fee for,
+warranty, support, indemnity or liability obligations to one or more recipients
+of Covered Software. However, you may do so only on Your own behalf, and not on
+behalf of the Initial Developer or any Contributor. You must make it absolutely
+clear that any such warranty, support, indemnity or liability obligation is
+offered by You alone, and You hereby agree to indemnify the Initial Developer
+and every Contributor for any liability incurred by the Initial Developer or
+such Contributor as a result of warranty, support, indemnity or liability terms
+You offer.
+3.5. Distribution of Executable Versions.
+You may distribute the
+Executable form of the Covered Software under the terms of this License or under
+the terms of a license of Your choice, which may contain terms different from
+this License, provided that You are in compliance with the terms of this License
+and that the license for the Executable form does not attempt to limit or alter
+the recipient's rights in the Source Code form from the rights set forth in
+this License. If You distribute the Covered Software in Executable form under a
+different license, You must make it absolutely clear that any terms which differ
+from this License are offered by You alone, not by the Initial Developer or
+Contributor. You hereby agree to indemnify the Initial Developer and every
+Contributor for any liability incurred by the Initial Developer or such
+Contributor as a result of any such terms You offer.
+3.6. Larger Works.
+You
+may create a Larger Work by combining Covered Software with other code not
+governed by the terms of this License and distribute the Larger Work as a single
+product. In such a case, You must make sure the requirements of this License are
+fulfilled for the Covered Software.
+
+4. Versions of the License.
+
+4.1. New Versions.
+Oracle is the initial license steward and may publish revised and/or
+new versions of this License from time to time. Each version will be given a
+distinguishing version number. Except as provided in Section 4.3, no one other
+than the license steward has the right to modify this License.
+4.2. Effect of New Versions.
+You may always continue to use, distribute or otherwise make the
+Covered Software available under the terms of the version of the License under
+which You originally received the Covered Software. If the Initial Developer
+includes a notice in the Original Software prohibiting it from being distributed
+or otherwise made available under any subsequent version of the License, You
+must distribute and make the Covered Software available under the terms of the
+version of the License under which You originally received the Covered Software.
+Otherwise, You may also choose to use, distribute or otherwise make the Covered
+Software available under the terms of any subsequent version of the License
+published by the license steward.
+4.3. Modified Versions.
+When You are an
+Initial Developer and You want to create a new license for Your Original
+Software, You may create and use a modified version of this License if You: (a)
+rename the license and remove any references to the name of the license steward
+(except to note that the license differs from this License); and (b) otherwise
+make it clear that the license contains terms which differ from this
+License.
+
+5. DISCLAIMER OF WARRANTY.
+
+COVERED SOFTWARE IS PROVIDED UNDER THIS
+LICENSE ON AN \u201cAS IS\u201d BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED
+OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE
+IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR
+NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED
+SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY
+RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE
+COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF
+WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED
+SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER.
+
+6. TERMINATION.
+
+6.1. This License and the rights granted hereunder will
+terminate automatically if You fail to comply with terms herein and fail to cure
+such breach within 30 days of becoming aware of the breach. Provisions which, by
+their nature, must remain in effect beyond the termination of this License shall
+survive.
+6.2. If You assert a patent infringement claim (excluding declaratory
+judgment actions) against Initial Developer or a Contributor (the Initial
+Developer or Contributor against whom You assert such claim is referred to as
+\u201cParticipant\u201d) alleging that the Participant Software (meaning the
+Contributor Version where the Participant is a Contributor or the Original
+Software where the Participant is the Initial Developer) directly or indirectly
+infringes any patent, then any and all rights granted directly or indirectly to
+You by such Participant, the Initial Developer (if the Initial Developer is not
+the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this
+License shall, upon 60 days notice from Participant terminate prospectively and
+automatically at the expiration of such 60 day notice period, unless if within
+such 60 day period You withdraw Your claim with respect to the Participant
+Software against such Participant either unilaterally or pursuant to a written
+agreement with Participant.
+6.3. If You assert a patent infringement claim
+against Participant alleging that the Participant Software directly or
+indirectly infringes any patent where such claim is resolved (such as by license
+or settlement) prior to the initiation of patent infringement litigation, then
+the reasonable value of the licenses granted by such Participant under Sections
+2.1 or 2.2 shall be taken into account in determining the amount or value of any
+payment or license.
+6.4. In the event of termination under Sections 6.1 or 6.2
+above, all end user licenses that have been validly granted by You or any
+distributor hereunder prior to termination (excluding licenses granted to You by
+any distributor) shall survive termination.
+
+7. LIMITATION OF LIABILITY.
+
+UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT
+(INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL
+DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY
+SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT,
+SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING,
+WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER
+FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN
+IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS
+LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL
+INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW
+PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR
+LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND
+LIMITATION MAY NOT APPLY TO YOU.
+
+8. U.S. GOVERNMENT END USERS.
+
+The Covered
+Software is a \u201ccommercial item,\u201d as that term is defined in 48 C.F.R. 2.101
+(Oct. 1995), consisting of \u201ccommercial computer software\u201d (as that term is
+defined at 48 C.F.R. � 252.227-7014(a)(1)) and \u201ccommercial computer software
+documentation\u201d as such terms are used in 48 C.F.R. 12.212 (Sept. 1995).
+Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4
+(June 1995), all U.S. Government End Users acquire Covered Software with only
+those rights set forth herein. This U.S. Government Rights clause is in lieu of,
+and supersedes, any other FAR, DFAR, or other clause or provision that addresses
+Government rights in computer software under this License.
+
+9. MISCELLANEOUS.
+
+This License represents the complete agreement concerning
+subject matter hereof. If any provision of this License is held to be
+unenforceable, such provision shall be reformed only to the extent necessary to
+make it enforceable. This License shall be governed by the law of the
+jurisdiction specified in a notice contained within the Original Software
+(except to the extent applicable law, if any, provides otherwise), excluding
+such jurisdiction's conflict-of-law provisions. Any litigation relating to this
+License shall be subject to the jurisdiction of the courts located in the
+jurisdiction and venue specified in a notice contained within the Original
+Software, with the losing party responsible for costs, including, without
+limitation, court costs and reasonable attorneys' fees and expenses. The
+application of the United Nations Convention on Contracts for the International
+Sale of Goods is expressly excluded. Any law or regulation which provides that
+the language of a contract shall be construed against the drafter shall not
+apply to this License. You agree that You alone are responsible for compliance
+with the United States export administration regulations (and the export control
+laws and regulation of any other countries) when You use, distribute or
+otherwise make available any Covered Software.
+
+10. RESPONSIBILITY FOR CLAIMS.
+
+As between Initial Developer and the Contributors, each party is
+responsible for claims and damages arising, directly or indirectly, out of its
+utilization of rights under this License and You agree to work with Initial
+Developer and Contributors to distribute such responsibility on an equitable
+basis. Nothing herein is intended or shall be deemed to constitute any admission
+of liability.
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+Protocol Buffer Java API 2.5.0
+--------------------------------------------------------------------------------
+This license applies to all parts of Protocol Buffers except the following:
+
+  - Atomicops support for generic gcc, located in
+    src/google/protobuf/stubs/atomicops_internals_generic_gcc.h.
+    This file is copyrighted by Red Hat Inc.
+
+  - Atomicops support for AIX/POWER, located in
+    src/google/protobuf/stubs/atomicops_internals_power.h.
+    This file is copyrighted by Bloomberg Finance LP.
+
+Copyright 2014, Google Inc.  All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+    * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+Code generated by the Protocol Buffer compiler is owned by the owner
+of the input file used when generating it.  This code is not
+standalone and requires a support library to be linked with it.  This
+support library is itself covered by the above license.
+
+For:
+XML Commons External Components XML APIs 1.3.04
+--------------------------------------------------------------------------------
+By obtaining, using and/or copying this work, you (the licensee) agree that you
+have read, understood, and will comply with the following terms and conditions.
+
+Permission to copy, modify, and distribute this software and its documentation,
+with or without modification, for any purpose and without fee or royalty is
+hereby granted, provided that you include the following on ALL copies of the
+software and documentation or portions thereof, including modifications:
+- The full text of this NOTICE in a location viewable to users of the
+redistributed or derivative work.
+- Any pre-existing intellectual property disclaimers, notices, or terms and
+conditions. If none exist, the W3C Software Short Notice should be included
+(hypertext is preferred, text is permitted) within the body of any redistributed
+or derivative code.
+- Notice of any changes or modifications to the files, including the date changes
+were made. (We recommend you provide URIs to the location from which the code is
+derived.)
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+JUnit 4.11
+ecj-4.3.1.jar
+--------------------------------------------------------------------------------
+Eclipse Public License - v 1.0
+
+THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC
+LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM
+CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
+
+1. DEFINITIONS
+
+"Contribution" means:
+
+a) in the case of the initial Contributor, the initial code and documentation
+distributed under this Agreement, and
+b) in the case of each subsequent Contributor:
+i) changes to the Program, and
+ii) additions to the Program;
+where such changes and/or additions to the Program originate from and are
+distributed by that particular Contributor. A Contribution 'originates' from a
+Contributor if it was added to the Program by such Contributor itself or anyone
+acting on such Contributor's behalf. Contributions do not include additions to
+the Program which: (i) are separate modules of software distributed in
+conjunction with the Program under their own license agreement, and (ii) are not
+derivative works of the Program.
+"Contributor" means any person or entity that distributes the Program.
+
+"Licensed Patents" mean patent claims licensable by a Contributor which are
+necessarily infringed by the use or sale of its Contribution alone or when
+combined with the Program.
+
+"Program" means the Contributions distributed in accordance with this Agreement.
+
+"Recipient" means anyone who receives the Program under this Agreement,
+including all Contributors.
+
+2. GRANT OF RIGHTS
+
+a) Subject to the terms of this Agreement, each Contributor hereby grants
+Recipient a non-exclusive, worldwide, royalty-free copyright license to
+reproduce, prepare derivative works of, publicly display, publicly perform,
+distribute and sublicense the Contribution of such Contributor, if any, and such
+derivative works, in source code and object code form.
+b) Subject to the terms of this Agreement, each Contributor hereby grants
+Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed
+Patents to make, use, sell, offer to sell, import and otherwise transfer the
+Contribution of such Contributor, if any, in source code and object code form.
+This patent license shall apply to the combination of the Contribution and the
+Program if, at the time the Contribution is added by the Contributor, such
+addition of the Contribution causes such combination to be covered by the
+Licensed Patents. The patent license shall not apply to any other combinations
+which include the Contribution. No hardware per se is licensed hereunder.
+c) Recipient understands that although each Contributor grants the licenses to
+its Contributions set forth herein, no assurances are provided by any
+Contributor that the Program does not infringe the patent or other intellectual
+property rights of any other entity. Each Contributor disclaims any liability to
+Recipient for claims brought by any other entity based on infringement of
+intellectual property rights or otherwise. As a condition to exercising the
+rights and licenses granted hereunder, each Recipient hereby assumes sole
+responsibility to secure any other intellectual property rights needed, if any.
+For example, if a third party patent license is required to allow Recipient to
+distribute the Program, it is Recipient's responsibility to acquire that license
+before distributing the Program.
+d) Each Contributor represents that to its knowledge it has sufficient copyright
+rights in its Contribution, if any, to grant the copyright license set forth in
+this Agreement.
+3. REQUIREMENTS
+
+A Contributor may choose to distribute the Program in object code form under its
+own license agreement, provided that:
+
+a) it complies with the terms and conditions of this Agreement; and
+b) its license agreement:
+i) effectively disclaims on behalf of all Contributors all warranties and
+conditions, express and implied, including warranties or conditions of title and
+non-infringement, and implied warranties or conditions of merchantability and
+fitness for a particular purpose;
+ii) effectively excludes on behalf of all Contributors all liability for
+damages, including direct, indirect, special, incidental and consequential
+damages, such as lost profits;
+iii) states that any provisions which differ from this Agreement are offered by
+that Contributor alone and not by any other party; and
+iv) states that source code for the Program is available from such Contributor,
+and informs licensees how to obtain it in a reasonable manner on or through a
+medium customarily used for software exchange.
+When the Program is made available in source code form:
+
+a) it must be made available under this Agreement; and
+b) a copy of this Agreement must be included with each copy of the Program.
+Contributors may not remove or alter any copyright notices contained within the
+Program.
+
+Each Contributor must identify itself as the originator of its Contribution, if
+any, in a manner that reasonably allows subsequent Recipients to identify the
+originator of the Contribution.
+
+4. COMMERCIAL DISTRIBUTION
+
+Commercial distributors of software may accept certain responsibilities with
+respect to end users, business partners and the like. While this license is
+intended to facilitate the commercial use of the Program, the Contributor who
+includes the Program in a commercial product offering should do so in a manner
+which does not create potential liability for other Contributors. Therefore, if
+a Contributor includes the Program in a commercial product offering, such
+Contributor ("Commercial Contributor") hereby agrees to defend and indemnify
+every other Contributor ("Indemnified Contributor") against any losses, damages
+and costs (collectively "Losses") arising from claims, lawsuits and other legal
+actions brought by a third party against the Indemnified Contributor to the
+extent caused by the acts or omissions of such Commercial Contributor in
+connection with its distribution of the Program in a commercial product
+offering. The obligations in this section do not apply to any claims or Losses
+relating to any actual or alleged intellectual property infringement. In order
+to qualify, an Indemnified Contributor must: a) promptly notify the Commercial
+Contributor in writing of such claim, and b) allow the Commercial Contributor to
+control, and cooperate with the Commercial Contributor in, the defense and any
+related settlement negotiations. The Indemnified Contributor may participate in
+any such claim at its own expense.
+
+For example, a Contributor might include the Program in a commercial product
+offering, Product X. That Contributor is then a Commercial Contributor. If that
+Commercial Contributor then makes performance claims, or offers warranties
+related to Product X, those performance claims and warranties are such
+Commercial Contributor's responsibility alone. Under this section, the
+Commercial Contributor would have to defend claims against the other
+Contributors related to those performance claims and warranties, and if a court
+requires any other Contributor to pay any damages as a result, the Commercial
+Contributor must pay those damages.
+
+5. NO WARRANTY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR
+IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE,
+NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each
+Recipient is solely responsible for determining the appropriateness of using and
+distributing the Program and assumes all risks associated with its exercise of
+rights under this Agreement , including but not limited to the risks and costs
+of program errors, compliance with applicable laws, damage to or loss of data,
+programs or equipment, and unavailability or interruption of operations.
+
+6. DISCLAIMER OF LIABILITY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY
+CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST
+PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY
+OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS
+GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+7. GENERAL
+
+If any provision of this Agreement is invalid or unenforceable under applicable
+law, it shall not affect the validity or enforceability of the remainder of the
+terms of this Agreement, and without further action by the parties hereto, such
+provision shall be reformed to the minimum extent necessary to make such
+provision valid and enforceable.
+
+If Recipient institutes patent litigation against any entity (including a
+cross-claim or counterclaim in a lawsuit) alleging that the Program itself
+(excluding combinations of the Program with other software or hardware)
+infringes such Recipient's patent(s), then such Recipient's rights granted under
+Section 2(b) shall terminate as of the date such litigation is filed.
+
+All Recipient's rights under this Agreement shall terminate if it fails to
+comply with any of the material terms or conditions of this Agreement and does
+not cure such failure in a reasonable period of time after becoming aware of
+such noncompliance. If all Recipient's rights under this Agreement terminate,
+Recipient agrees to cease use and distribution of the Program as soon as
+reasonably practicable. However, Recipient's obligations under this Agreement
+and any licenses granted by Recipient relating to the Program shall continue and
+survive.
+
+Everyone is permitted to copy and distribute copies of this Agreement, but in
+order to avoid inconsistency the Agreement is copyrighted and may only be
+modified in the following manner. The Agreement Steward reserves the right to
+publish new versions (including revisions) of this Agreement from time to time.
+No one other than the Agreement Steward has the right to modify this Agreement.
+The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation
+may assign the responsibility to serve as the Agreement Steward to a suitable
+separate entity. Each new version of the Agreement will be given a
+distinguishing version number. The Program (including Contributions) may always
+be distributed subject to the version of the Agreement under which it was
+received. In addition, after a new version of the Agreement is published,
+Contributor may elect to distribute the Program (including its Contributions)
+under the new version. Except as expressly stated in Sections 2(a) and 2(b)
+above, Recipient receives no rights or licenses to the intellectual property of
+any Contributor under this Agreement, whether expressly, by implication,
+estoppel or otherwise. All rights in the Program not expressly granted under
+this Agreement are reserved.
+
+This Agreement is governed by the laws of the State of New York and the
+intellectual property laws of the United States of America. No party to this
+Agreement will bring a legal action under this Agreement more than one year
+after the cause of action arose. Each party waives its rights to a jury trial in
+any resulting litigation.
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+ASM Core 3.2
+JSch 0.1.51
+ParaNamer Core 2.3
+JLine 0.9.94
+leveldbjni-all 1.8
+Hamcrest Core 1.3
+xmlenc Library 0.52
+--------------------------------------------------------------------------------
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+    * Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above copyright
+      notice, this list of conditions and the following disclaimer in the
+      documentation and/or other materials provided with the distribution.
+    * Neither the name of the <organization> nor the
+      names of its contributors may be used to endorse or promote products
+      derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL <COPYRIGHT HOLDER> BE LIABLE FOR ANY
+DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+FindBugs-jsr305 3.0.0
+--------------------------------------------------------------------------------
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+1. Redistributions of source code must retain the above copyright notice, this
+   list of conditions and the following disclaimer.
+2. Redistributions in binary form must reproduce the above copyright notice,
+   this list of conditions and the following disclaimer in the documentation
+   and/or other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+The views and conclusions contained in the software and documentation are those
+of the authors and should not be interpreted as representing official policies,
+either expressed or implied, of the FreeBSD Project.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e383b732/NOTICE.txt
----------------------------------------------------------------------
diff --git a/NOTICE.txt b/NOTICE.txt
index efa865f..63fbc9d 100644
--- a/NOTICE.txt
+++ b/NOTICE.txt
@@ -15,3 +15,269 @@ which has the following notices:
 * This product includes software developed at
   Progress Software Corporation and/or its  subsidiaries or affiliates.
 * This product includes software developed by IBM Corporation and others.
+
+The binary distribution of this product bundles binaries of
+AWS Java SDK 1.10.6,
+which has the following notices:
+ * This software includes third party software subject to the following
+ copyrights: - XML parsing and utility functions from JetS3t - Copyright
+ 2006-2009 James Murty. - JSON parsing and utility functions from JSON.org -
+ Copyright 2002 JSON.org. - PKCS#1 PEM encoded private key parsing and utility
+ functions from oauth.googlecode.com - Copyright 1998-2010 AOL Inc.
+
+The binary distribution of this product bundles binaries of
+Gson 2.2.4,
+which has the following notices:
+
+                            The Netty Project
+                            =================
+
+Please visit the Netty web site for more information:
+
+  * http://netty.io/
+
+Copyright 2014 The Netty Project
+
+The Netty Project 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.
+
+Also, please refer to each LICENSE.<component>.txt file, which is located in
+the 'license' directory of the distribution file, for the license terms of the
+components that this product depends on.
+
+-------------------------------------------------------------------------------
+This product contains the extensions to Java Collections Framework which has
+been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
+
+  * LICENSE:
+    * license/LICENSE.jsr166y.txt (Public Domain)
+  * HOMEPAGE:
+    * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
+    * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
+
+This product contains a modified version of Robert Harder's Public Domain
+Base64 Encoder and Decoder, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.base64.txt (Public Domain)
+  * HOMEPAGE:
+    * http://iharder.sourceforge.net/current/java/base64/
+
+This product contains a modified portion of 'Webbit', an event based
+WebSocket and HTTP server, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.webbit.txt (BSD License)
+  * HOMEPAGE:
+    * https://github.com/joewalnes/webbit
+
+This product contains a modified portion of 'SLF4J', a simple logging
+facade for Java, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.slf4j.txt (MIT License)
+  * HOMEPAGE:
+    * http://www.slf4j.org/
+
+This product contains a modified portion of 'ArrayDeque', written by Josh
+Bloch of Google, Inc:
+
+  * LICENSE:
+    * license/LICENSE.deque.txt (Public Domain)
+
+This product contains a modified portion of 'Apache Harmony', an open source
+Java SE, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.harmony.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://archive.apache.org/dist/harmony/
+
+This product contains a modified version of Roland Kuhn's ASL2
+AbstractNodeQueue, which is based on Dmitriy Vyukov's non-intrusive MPSC queue.
+It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.abstractnodequeue.txt (Public Domain)
+  * HOMEPAGE:
+    * https://github.com/akka/akka/blob/wip-2.2.3-for-scala-2.11/akka-actor/src/main/java/akka/dispatch/AbstractNodeQueue.java
+
+This product contains a modified portion of 'jbzip2', a Java bzip2 compression
+and decompression library written by Matthew J. Francis. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jbzip2.txt (MIT License)
+  * HOMEPAGE:
+    * https://code.google.com/p/jbzip2/
+
+This product contains a modified portion of 'libdivsufsort', a C API library to construct
+the suffix array and the Burrows-Wheeler transformed string for any input string of
+a constant-size alphabet written by Yuta Mori. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.libdivsufsort.txt (MIT License)
+  * HOMEPAGE:
+    * https://code.google.com/p/libdivsufsort/
+
+This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
+ which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jctools.txt (ASL2 License)
+  * HOMEPAGE:
+    * https://github.com/JCTools/JCTools
+
+This product optionally depends on 'JZlib', a re-implementation of zlib in
+pure Java, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jzlib.txt (BSD style License)
+  * HOMEPAGE:
+    * http://www.jcraft.com/jzlib/
+
+This product optionally depends on 'Compress-LZF', a Java library for encoding and
+decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.compress-lzf.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/ning/compress
+
+This product optionally depends on 'lz4', a LZ4 Java compression
+and decompression library written by Adrien Grand. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.lz4.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/jpountz/lz4-java
+
+This product optionally depends on 'lzma-java', a LZMA Java compression
+and decompression library, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.lzma-java.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/jponge/lzma-java
+
+This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
+and decompression library written by William Kinney. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jfastlz.txt (MIT License)
+  * HOMEPAGE:
+    * https://code.google.com/p/jfastlz/
+
+This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
+interchange format, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.protobuf.txt (New BSD License)
+  * HOMEPAGE:
+    * http://code.google.com/p/protobuf/
+
+This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
+a temporary self-signed X.509 certificate when the JVM does not provide the
+equivalent functionality.  It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.bouncycastle.txt (MIT License)
+  * HOMEPAGE:
+    * http://www.bouncycastle.org/
+
+This product optionally depends on 'Snappy', a compression library produced
+by Google Inc, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.snappy.txt (New BSD License)
+  * HOMEPAGE:
+    * http://code.google.com/p/snappy/
+
+This product optionally depends on 'JBoss Marshalling', an alternative Java
+serialization API, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1)
+  * HOMEPAGE:
+    * http://www.jboss.org/jbossmarshalling
+
+This product optionally depends on 'Caliper', Google's micro-
+benchmarking framework, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.caliper.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://code.google.com/p/caliper/
+
+This product optionally depends on 'Apache Commons Logging', a logging
+framework, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.commons-logging.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://commons.apache.org/logging/
+
+This product optionally depends on 'Apache Log4J', a logging framework, which
+can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.log4j.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://logging.apache.org/log4j/
+
+This product optionally depends on 'Aalto XML', an ultra-high performance
+non-blocking XML processor, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.aalto-xml.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://wiki.fasterxml.com/AaltoHome
+
+This product contains a modified version of 'HPACK', a Java implementation of
+the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.hpack.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/twitter/hpack
+
+This product contains a modified portion of 'Apache Commons Lang', a Java library
+provides utilities for the java.lang API, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.commons-lang.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://commons.apache.org/proper/commons-lang/
+
+The binary distribution of this product bundles binaries of
+Commons Codec 1.4,
+which has the following notices:
+ * src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.javacontains test data from http://aspell.net/test/orig/batch0.tab.Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org)
+  ===============================================================================
+  The content of package org.apache.commons.codec.language.bm has been translated
+  from the original php source code available at http://stevemorse.org/phoneticinfo.htm
+  with permission from the original authors.
+  Original source copyright:Copyright (c) 2008 Alexander Beider & Stephen P. Morse.
+
+The binary distribution of this product bundles binaries of
+Commons Lang 2.6,
+which has the following notices:
+ * This product includes software from the Spring Framework,under the Apache License 2.0 (see: StringUtils.containsWhitespace())
+
+The binary distribution of this product bundles binaries of
+Apache Log4j 1.2.17,
+which has the following notices:
+ * ResolverUtil.java
+    Copyright 2005-2006 Tim Fennell
+  Dumbster SMTP test server
+    Copyright 2004 Jason Paul Kitchen
+  TypeUtil.java
+    Copyright 2002-2012 Ramnivas Laddad, Juergen Hoeller, Chris Beams
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e383b732/hadoop-build-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-build-tools/pom.xml b/hadoop-build-tools/pom.xml
index d08cb89..e84c94f 100644
--- a/hadoop-build-tools/pom.xml
+++ b/hadoop-build-tools/pom.xml
@@ -28,4 +28,45 @@
   <properties>
     <failIfNoTests>false</failIfNoTests>
   </properties>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-site-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-remote-resources-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>bundle</goal>
+            </goals>
+          </execution>
+        </executions>
+        <configuration>
+          <includes>
+            <include>META-INF/LICENSE.txt</include>
+            <include>META-INF/NOTICE.txt</include>
+          </includes>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>dummy</id>
+            <phase>validate</phase>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e383b732/hadoop-project-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project-dist/pom.xml b/hadoop-project-dist/pom.xml
index 2cccbbe..ba4c0a8 100644
--- a/hadoop-project-dist/pom.xml
+++ b/hadoop-project-dist/pom.xml
@@ -80,6 +80,8 @@
               <includes>
                 <include>**/*.class</include>
                 <include>webapps/**</include>
+                <include>META-INF/LICENSE.txt</include>
+                <include>META-INF/NOTICE.txt</include>
               </includes>
             </configuration>
           </execution>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e383b732/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 35166b1..7558b04 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -111,6 +111,7 @@
     <maven-source-plugin.version>2.3</maven-source-plugin.version>
     <maven-project-info-reports-plugin.version>2.7</maven-project-info-reports-plugin.version>
     <maven-pdf-plugin.version>1.2</maven-pdf-plugin.version>
+    <maven-remote-resources-plugin.version>1.5</maven-remote-resources-plugin.version>
     <build-helper-maven-plugin.version>1.9</build-helper-maven-plugin.version>
     <exec-maven-plugin.version>1.3.1</exec-maven-plugin.version>
     <make-maven-plugin.version>1.0-beta-1</make-maven-plugin.version>
@@ -1134,7 +1135,23 @@
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>findbugs-maven-plugin</artifactId>
       </plugin>
-      
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-remote-resources-plugin</artifactId>
+        <version>${maven-remote-resources-plugin.version}</version>
+        <configuration>
+          <resourceBundles>
+            <resourceBundle>org.apache.hadoop:hadoop-build-tools:${project.version}</resourceBundle>
+          </resourceBundles>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>process</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
        <plugin>
          <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-antrun-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e383b732/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index c908340..aec6722 100644
--- a/pom.xml
+++ b/pom.xml
@@ -412,6 +412,32 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
         <artifactId>dependency-check-maven</artifactId>
         <version>${dependency-check-maven.version}</version>
       </plugin>
+      <plugin>
+        <!-- Copy license and notice files into hadoop-build-tools'
+             metadata, which will then be bundled into jars.
+        -->
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>copy-files</id>
+            <phase>process-resources</phase>
+            <goals>
+              <goal>run</goal>
+            </goals>
+            <configuration>
+              <target>
+                <copy todir="${project.build.directory}/../hadoop-build-tools/src/main/resources/META-INF/">
+                  <fileset dir="${project.build.directory}/../">
+                    <include name="LICENSE.txt"/>
+                    <include name="NOTICE.txt"/>
+                  </fileset>
+                </copy>
+              </target>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[10/51] [abbrv] hadoop git commit: Remove redundant TestMiniDFSCluster.testDualClusters. Contributed by Jiayi Zhou.

Posted by vv...@apache.org.
Remove redundant TestMiniDFSCluster.testDualClusters. Contributed by Jiayi Zhou.


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

Branch: refs/heads/YARN-3926
Commit: 1ee9ea002609971ad58082bf525d57fca8a37035
Parents: 5a43583
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Jun 8 12:58:56 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed Jun 8 12:58:56 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/TestMiniDFSCluster.java  | 31 --------------------
 1 file changed, 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ee9ea00/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
index 78ae8b1..ec72d87 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assume.assumeTrue;
 
 import java.io.File;
@@ -84,36 +83,6 @@ public class TestMiniDFSCluster {
     }
   }
 
-  /**
-   * Bring up two clusters and assert that they are in different directories.
-   * @throws Throwable on a failure
-   */
-  @Test(timeout=100000)
-  public void testDualClusters() throws Throwable {
-    File testDataCluster2 = new File(testDataPath, CLUSTER_2);
-    File testDataCluster3 = new File(testDataPath, CLUSTER_3);
-    Configuration conf = new HdfsConfiguration();
-    String c2Path = testDataCluster2.getAbsolutePath();
-    conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, c2Path);
-    MiniDFSCluster cluster2 = new MiniDFSCluster.Builder(conf).build();
-    MiniDFSCluster cluster3 = null;
-    try {
-      String dataDir2 = cluster2.getDataDirectory();
-      assertEquals(new File(c2Path + "/data"), new File(dataDir2));
-      //change the data dir
-      conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR,
-               testDataCluster3.getAbsolutePath());
-      MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
-      cluster3 = builder.build();
-      String dataDir3 = cluster3.getDataDirectory();
-      assertTrue("Clusters are bound to the same directory: " + dataDir2,
-                        !dataDir2.equals(dataDir3));
-    } finally {
-      MiniDFSCluster.shutdownCluster(cluster3);
-      MiniDFSCluster.shutdownCluster(cluster2);
-    }
-  }
-
   @Test(timeout=100000)
   public void testIsClusterUpAfterShutdown() throws Throwable {
     Configuration conf = new HdfsConfiguration();


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[16/51] [abbrv] hadoop git commit: HADOOP-13237: s3a initialization against public bucket fails if caller lacks any credentials. Contributed by Chris Nauroth

Posted by vv...@apache.org.
HADOOP-13237: s3a initialization against public bucket fails if caller lacks any credentials. Contributed by Chris Nauroth


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

Branch: refs/heads/YARN-3926
Commit: 656c460c0e79ee144d6ef48d85cec04a1af3b2cc
Parents: 8ea9bbc
Author: Steve Loughran <st...@apache.org>
Authored: Thu Jun 9 16:36:27 2016 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Jun 9 17:28:49 2016 +0100

----------------------------------------------------------------------
 .../src/main/resources/core-default.xml         | 13 ++++-
 .../fs/s3a/AnonymousAWSCredentialsProvider.java | 11 ++++
 .../fs/s3a/BasicAWSCredentialsProvider.java     |  8 +++
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 22 +++++---
 .../src/site/markdown/tools/hadoop-aws/index.md | 14 ++++-
 .../fs/s3a/TestS3AAWSCredentialsProvider.java   | 55 ++++++++++++++++++++
 6 files changed, 113 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/656c460c/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index a65246b..8bb27ea 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -791,7 +791,18 @@
 
 <property>
   <name>fs.s3a.aws.credentials.provider</name>
-  <description>Class name of a credentials provider that implements com.amazonaws.auth.AWSCredentialsProvider. Omit if using access/secret keys or another authentication mechanism.</description>
+  <description>
+    Class name of a credentials provider that implements
+    com.amazonaws.auth.AWSCredentialsProvider.  Omit if using access/secret keys
+    or another authentication mechanism.  The specified class must provide an
+    accessible constructor accepting java.net.URI and
+    org.apache.hadoop.conf.Configuration, or an accessible default constructor.
+    Specifying org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider allows
+    anonymous access to a publicly accessible S3 bucket without any credentials.
+    Please note that allowing anonymous access to an S3 bucket compromises
+    security and therefore is unsuitable for most use cases.  It can be useful
+    for accessing public data sets without requiring AWS credentials.
+  </description>
 </property>
 
 <property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/656c460c/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AnonymousAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AnonymousAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AnonymousAWSCredentialsProvider.java
index e62ec77..2c863fc 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AnonymousAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/AnonymousAWSCredentialsProvider.java
@@ -24,6 +24,17 @@ import com.amazonaws.auth.AWSCredentials;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
+/**
+ * AnonymousAWSCredentialsProvider supports anonymous access to AWS services
+ * through the AWS SDK.  AWS requests will not be signed.  This is not suitable
+ * for most cases, because allowing anonymous access to an S3 bucket compromises
+ * security.  This can be useful for accessing public data sets without
+ * requiring AWS credentials.
+ *
+ * Please note that users may reference this class name from configuration
+ * property fs.s3a.aws.credentials.provider.  Therefore, changing the class name
+ * would be a backward-incompatible change.
+ */
 @InterfaceAudience.Private
 @InterfaceStability.Stable
 public class AnonymousAWSCredentialsProvider implements AWSCredentialsProvider {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/656c460c/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
index 2f721e4..3a5ee8c 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
@@ -26,6 +26,14 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
+/**
+ * BasicAWSCredentialsProvider supports static configuration of access key ID
+ * and secret access key for use with the AWS SDK.
+ *
+ * Please note that users may reference this class name from configuration
+ * property fs.s3a.aws.credentials.provider.  Therefore, changing the class name
+ * would be a backward-incompatible change.
+ */
 @InterfaceAudience.Private
 @InterfaceStability.Stable
 public class BasicAWSCredentialsProvider implements AWSCredentialsProvider {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/656c460c/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 0281a3a..9af0a99 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
@@ -465,20 +465,28 @@ public class S3AFileSystem extends FileSystem {
           new BasicAWSCredentialsProvider(
               creds.getAccessKey(), creds.getAccessSecret()),
           new InstanceProfileCredentialsProvider(),
-          new EnvironmentVariableCredentialsProvider(),
-          new AnonymousAWSCredentialsProvider()
-      );
+          new EnvironmentVariableCredentialsProvider());
 
     } else {
       try {
         LOG.debug("Credential provider class is {}", className);
-        credentials = (AWSCredentialsProvider) Class.forName(className)
-            .getDeclaredConstructor(URI.class, Configuration.class)
-            .newInstance(this.uri, conf);
+        Class<?> credClass = Class.forName(className);
+        try {
+          credentials =
+              (AWSCredentialsProvider)credClass.getDeclaredConstructor(
+                  URI.class, Configuration.class).newInstance(this.uri, conf);
+        } catch (NoSuchMethodException | SecurityException e) {
+          credentials =
+              (AWSCredentialsProvider)credClass.getDeclaredConstructor()
+                  .newInstance();
+        }
       } catch (ClassNotFoundException e) {
         throw new IOException(className + " not found.", e);
       } catch (NoSuchMethodException | SecurityException e) {
-        throw new IOException(className + " constructor exception.", e);
+        throw new IOException(String.format("%s constructor exception.  A "
+            + "class specified in %s must provide an accessible constructor "
+            + "accepting URI and Configuration, or an accessible default "
+            + "constructor.", className, AWS_CREDENTIALS_PROVIDER), e);
       } catch (ReflectiveOperationException | IllegalArgumentException e) {
         throw new IOException(className + " instantiation exception.", e);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/656c460c/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index 7d63a86..4086bc0 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -187,8 +187,18 @@ If you do any of these: change your credentials immediately!
 
     <property>
       <name>fs.s3a.aws.credentials.provider</name>
-      <description>Class name of a credentials provider that implements com.amazonaws.auth.AWSCredentialsProvider.
-      Omit if using access/secret keys or another authentication mechanism.</description>
+      <description>
+        Class name of a credentials provider that implements
+        com.amazonaws.auth.AWSCredentialsProvider.  Omit if using access/secret keys
+        or another authentication mechanism.  The specified class must provide an
+        accessible constructor accepting java.net.URI and
+        org.apache.hadoop.conf.Configuration, or an accessible default constructor.
+        Specifying org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider allows
+        anonymous access to a publicly accessible S3 bucket without any credentials.
+        Please note that allowing anonymous access to an S3 bucket compromises
+        security and therefore is unsuitable for most use cases.  It can be useful
+        for accessing public data sets without requiring AWS credentials.
+      </description>
     </property>
 
 #### Protecting the AWS Credentials in S3A

http://git-wip-us.apache.org/repos/asf/hadoop/blob/656c460c/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
index 1a11a45..a25ca9c 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.s3a;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
 import static org.junit.Assert.*;
 
 import java.io.IOException;
@@ -26,8 +27,13 @@ import java.net.URI;
 import java.nio.file.AccessDeniedException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.Timeout;
 
 import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.auth.AWSCredentialsProvider;
@@ -45,6 +51,12 @@ public class TestS3AAWSCredentialsProvider {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestS3AAWSCredentialsProvider.class);
 
+  @Rule
+  public Timeout testTimeout = new Timeout(1 * 60 * 1000);
+
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
   @Test
   public void testBadConfiguration() throws IOException {
     Configuration conf = new Configuration();
@@ -113,4 +125,47 @@ public class TestS3AAWSCredentialsProvider {
     conf.set(AWS_CREDENTIALS_PROVIDER, GoodCredentialsProvider.class.getName());
     S3ATestUtils.createTestFileSystem(conf);
   }
+
+  @Test
+  public void testAnonymousProvider() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(AWS_CREDENTIALS_PROVIDER,
+        AnonymousAWSCredentialsProvider.class.getName());
+    Path testFile = new Path(
+        conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
+    FileSystem fs = FileSystem.newInstance(testFile.toUri(), conf);
+    assertNotNull(fs);
+    assertTrue(fs instanceof S3AFileSystem);
+    FileStatus stat = fs.getFileStatus(testFile);
+    assertNotNull(stat);
+    assertEquals(testFile, stat.getPath());
+  }
+
+  static class ConstructorErrorProvider implements AWSCredentialsProvider {
+
+    @SuppressWarnings("unused")
+    public ConstructorErrorProvider(String str) {
+    }
+
+    @Override
+    public AWSCredentials getCredentials() {
+      return null;
+    }
+
+    @Override
+    public void refresh() {
+    }
+  }
+
+  @Test
+  public void testProviderConstructorError() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(AWS_CREDENTIALS_PROVIDER,
+        ConstructorErrorProvider.class.getName());
+    Path testFile = new Path(
+        conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
+    exception.expect(IOException.class);
+    exception.expectMessage("constructor exception");
+    FileSystem fs = FileSystem.newInstance(testFile.toUri(), conf);
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[35/51] [abbrv] hadoop git commit: YARN-3426. Add jdiff support to YARN. (vinodkv via wangda)

Posted by vv...@apache.org.
YARN-3426. Add jdiff support to YARN. (vinodkv via wangda)


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

Branch: refs/heads/YARN-3926
Commit: 03fc6b1bb0f5c0844cd5477ffba43de8a14d4d60
Parents: 244506f
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Jun 10 09:51:09 2016 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Jun 10 09:51:09 2016 -0700

----------------------------------------------------------------------
 .../classification/tools/RootDocProcessor.java  |     4 +
 .../jdiff/Apache_Hadoop_YARN_API_2.6.0.xml      | 13076 ++++++++++++++++
 .../jdiff/Apache_Hadoop_YARN_API_2.7.2.xml      | 13692 +++++++++++++++++
 .../jdiff/Apache_Hadoop_YARN_Client.2.6.0.xml   |  2427 +++
 .../jdiff/Apache_Hadoop_YARN_Client_2.7.2.xml   |  2581 ++++
 .../jdiff/Apache_Hadoop_YARN_Common_2.6.0.xml   |  2870 ++++
 .../jdiff/Apache_Hadoop_YARN_Common_2.7.2.xml   |  3323 ++++
 .../Apache_Hadoop_YARN_Server_Common_2.6.0.xml  |  2059 +++
 .../Apache_Hadoop_YARN_Server_Common_2.7.2.xml  |  1801 +++
 .../hadoop-yarn/dev-support/jdiff/Null.java     |    20 +
 .../hadoop-yarn/hadoop-yarn-api/pom.xml         |     2 +
 .../hadoop-yarn/hadoop-yarn-client/pom.xml      |     2 +
 .../hadoop-yarn/hadoop-yarn-common/pom.xml      |     2 +
 .../hadoop-yarn-server-common/pom.xml           |     2 +
 hadoop-yarn-project/hadoop-yarn/pom.xml         |   129 +
 15 files changed, 41990 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/03fc6b1b/hadoop-common-project/hadoop-annotations/src/main/java/org/apache/hadoop/classification/tools/RootDocProcessor.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-annotations/src/main/java/org/apache/hadoop/classification/tools/RootDocProcessor.java b/hadoop-common-project/hadoop-annotations/src/main/java/org/apache/hadoop/classification/tools/RootDocProcessor.java
index 8042f17..60c2a6f 100644
--- a/hadoop-common-project/hadoop-annotations/src/main/java/org/apache/hadoop/classification/tools/RootDocProcessor.java
+++ b/hadoop-common-project/hadoop-annotations/src/main/java/org/apache/hadoop/classification/tools/RootDocProcessor.java
@@ -127,6 +127,10 @@ class RootDocProcessor {
               return filter(((ClassDoc) target).constructors(true),
                   ConstructorDoc.class);
             }
+          } else {
+            if (methodName.equals("methods")) {
+              return filter(((ClassDoc) target).methods(true), MethodDoc.class);
+            }
           }
         } else if (target instanceof PackageDoc) {
           if (methodName.equals("allClasses")) {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[38/51] [abbrv] hadoop git commit: YARN-5197. RM leaks containers if running container disappears from node update. Contributed by Jason Lowe.

Posted by vv...@apache.org.
YARN-5197. RM leaks containers if running container disappears from node update. Contributed by Jason Lowe.


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

Branch: refs/heads/YARN-3926
Commit: e0f4620cc7db3db4b781e6042ab7dd754af28f18
Parents: 8a1dcce
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Sat Jun 11 10:22:27 2016 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Sat Jun 11 10:22:27 2016 +0530

----------------------------------------------------------------------
 .../resourcemanager/rmnode/RMNodeImpl.java      | 37 +++++++++++++
 .../yarn/server/resourcemanager/MockNM.java     | 57 ++++++++++++++------
 .../resourcemanager/TestRMNodeTransitions.java  | 44 +++++++++++++++
 3 files changed, 121 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0f4620c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.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/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
index 4b65675..a3a6b30 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
@@ -24,6 +24,7 @@ import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -72,6 +73,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRunningOnNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.AllocationExpirationInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeResourceUpdateSchedulerEvent;
@@ -1311,6 +1313,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
         new ArrayList<ContainerStatus>();
     List<ContainerStatus> completedContainers =
         new ArrayList<ContainerStatus>();
+    int numRemoteRunningContainers = 0;
     for (ContainerStatus remoteContainer : containerStatuses) {
       ContainerId containerId = remoteContainer.getContainerId();
 
@@ -1344,6 +1347,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       if (remoteContainer.getState() == ContainerState.RUNNING) {
         // Process only GUARANTEED containers in the RM.
         if (remoteContainer.getExecutionType() == ExecutionType.GUARANTEED) {
+          ++numRemoteRunningContainers;
           if (!launchedContainers.contains(containerId)) {
             // Just launched container. RM knows about it the first time.
             launchedContainers.add(containerId);
@@ -1366,12 +1370,45 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
         completedContainers.add(remoteContainer);
       }
     }
+    completedContainers.addAll(findLostContainers(
+          numRemoteRunningContainers, containerStatuses));
+
     if (newlyLaunchedContainers.size() != 0 || completedContainers.size() != 0) {
       nodeUpdateQueue.add(new UpdatedContainerInfo(newlyLaunchedContainers,
           completedContainers));
     }
   }
 
+  private List<ContainerStatus> findLostContainers(int numRemoteRunning,
+      List<ContainerStatus> containerStatuses) {
+    if (numRemoteRunning >= launchedContainers.size()) {
+      return Collections.emptyList();
+    }
+    Set<ContainerId> nodeContainers =
+        new HashSet<ContainerId>(numRemoteRunning);
+    List<ContainerStatus> lostContainers = new ArrayList<ContainerStatus>(
+        launchedContainers.size() - numRemoteRunning);
+    for (ContainerStatus remoteContainer : containerStatuses) {
+      if (remoteContainer.getState() == ContainerState.RUNNING
+          && remoteContainer.getExecutionType() == ExecutionType.GUARANTEED) {
+        nodeContainers.add(remoteContainer.getContainerId());
+      }
+    }
+    Iterator<ContainerId> iter = launchedContainers.iterator();
+    while (iter.hasNext()) {
+      ContainerId containerId = iter.next();
+      if (!nodeContainers.contains(containerId)) {
+        String diag = "Container " + containerId
+            + " was running but not reported from " + nodeId;
+        LOG.warn(diag);
+        lostContainers.add(SchedulerUtils.createAbnormalContainerStatus(
+            containerId, diag));
+        iter.remove();
+      }
+    }
+    return lostContainers;
+  }
+
   private void handleLogAggregationStatus(
       List<LogAggregationReport> logAggregationReportsForApps) {
     for (LogAggregationReport report : logAggregationReportsForApps) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0f4620c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.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/MockNM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
index 04ea51c..2e2bef7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
@@ -28,6 +28,7 @@ import java.util.Map;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -57,6 +58,8 @@ public class MockNM {
   private MasterKey currentContainerTokenMasterKey;
   private MasterKey currentNMTokenMasterKey;
   private String version;
+  private Map<ContainerId, ContainerStatus> containerStats =
+      new HashMap<ContainerId, ContainerStatus>();
 
   public MockNM(String nodeIdStr, int memory, ResourceTrackerService resourceTracker) {
     // scale vcores based on the requested memory
@@ -106,14 +109,12 @@ public class MockNM {
   }
 
   public void containerIncreaseStatus(Container container) throws Exception {
-    Map<ApplicationId, List<ContainerStatus>> conts = new HashMap<>();
     ContainerStatus containerStatus = BuilderUtils.newContainerStatus(
         container.getId(), ContainerState.RUNNING, "Success", 0,
             container.getResource());
-    conts.put(container.getId().getApplicationAttemptId().getApplicationId(),
-        Collections.singletonList(containerStatus));
     List<Container> increasedConts = Collections.singletonList(container);
-    nodeHeartbeat(conts, increasedConts, true, ++responseId);
+    nodeHeartbeat(Collections.singletonList(containerStatus), increasedConts,
+        true, ++responseId);
   }
 
   public RegisterNodeManagerResponse registerNode() throws Exception {
@@ -147,18 +148,27 @@ public class MockNM {
       memory = (int) newResource.getMemorySize();
       vCores = newResource.getVirtualCores();
     }
+    containerStats.clear();
+    if (containerReports != null) {
+      for (NMContainerStatus report : containerReports) {
+        if (report.getContainerState() != ContainerState.COMPLETE) {
+          containerStats.put(report.getContainerId(),
+              ContainerStatus.newInstance(report.getContainerId(),
+                  report.getContainerState(), report.getDiagnostics(),
+                  report.getContainerExitStatus()));
+        }
+      }
+    }
     return registrationResponse;
   }
 
   public NodeHeartbeatResponse nodeHeartbeat(boolean isHealthy) throws Exception {
-    return nodeHeartbeat(new HashMap<ApplicationId, List<ContainerStatus>>(),
-        isHealthy, ++responseId);
+    return nodeHeartbeat(Collections.<ContainerStatus>emptyList(),
+        Collections.<Container>emptyList(), isHealthy, ++responseId);
   }
 
   public NodeHeartbeatResponse nodeHeartbeat(ApplicationAttemptId attemptId,
       long containerId, ContainerState containerState) throws Exception {
-    HashMap<ApplicationId, List<ContainerStatus>> nodeUpdate =
-        new HashMap<ApplicationId, List<ContainerStatus>>(1);
     ContainerStatus containerStatus = BuilderUtils.newContainerStatus(
         BuilderUtils.newContainerId(attemptId, containerId), containerState,
         "Success", 0, BuilderUtils.newResource(memory, vCores));
@@ -166,8 +176,8 @@ public class MockNM {
         new ArrayList<ContainerStatus>(1);
     containerStatusList.add(containerStatus);
     Log.info("ContainerStatus: " + containerStatus);
-    nodeUpdate.put(attemptId.getApplicationId(), containerStatusList);
-    return nodeHeartbeat(nodeUpdate, true);
+    return nodeHeartbeat(containerStatusList,
+        Collections.<Container>emptyList(), true, ++responseId);
   }
 
   public NodeHeartbeatResponse nodeHeartbeat(Map<ApplicationId,
@@ -177,19 +187,32 @@ public class MockNM {
 
   public NodeHeartbeatResponse nodeHeartbeat(Map<ApplicationId,
       List<ContainerStatus>> conts, boolean isHealthy, int resId) throws Exception {
-    return nodeHeartbeat(conts, new ArrayList<Container>(), isHealthy, resId);
+    ArrayList<ContainerStatus> updatedStats = new ArrayList<ContainerStatus>();
+    for (List<ContainerStatus> stats : conts.values()) {
+      updatedStats.addAll(stats);
+    }
+    return nodeHeartbeat(updatedStats, Collections.<Container>emptyList(),
+        isHealthy, resId);
   }
 
-  public NodeHeartbeatResponse nodeHeartbeat(Map<ApplicationId,
-      List<ContainerStatus>> conts, List<Container> increasedConts,
-          boolean isHealthy, int resId) throws Exception {
+  public NodeHeartbeatResponse nodeHeartbeat(List<ContainerStatus> updatedStats,
+      List<Container> increasedConts, boolean isHealthy, int resId)
+          throws Exception {
     NodeHeartbeatRequest req = Records.newRecord(NodeHeartbeatRequest.class);
     NodeStatus status = Records.newRecord(NodeStatus.class);
     status.setResponseId(resId);
     status.setNodeId(nodeId);
-    for (Map.Entry<ApplicationId, List<ContainerStatus>> entry : conts.entrySet()) {
-      Log.info("entry.getValue() " + entry.getValue());
-      status.setContainersStatuses(entry.getValue());
+    ArrayList<ContainerId> completedContainers = new ArrayList<ContainerId>();
+    for (ContainerStatus stat : updatedStats) {
+      if (stat.getState() == ContainerState.COMPLETE) {
+        completedContainers.add(stat.getContainerId());
+      }
+      containerStats.put(stat.getContainerId(), stat);
+    }
+    status.setContainersStatuses(
+        new ArrayList<ContainerStatus>(containerStats.values()));
+    for (ContainerId cid : completedContainers) {
+      containerStats.remove(cid);
     }
     status.setIncreasedContainers(increasedConts);
     NodeHealthStatus healthStatus = Records.newRecord(NodeHealthStatus.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0f4620c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.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/TestRMNodeTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
index 16fe998..83a7c73 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
@@ -34,6 +34,7 @@ import java.util.Random;
 import org.apache.hadoop.util.HostsFileReader;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
@@ -1021,4 +1022,47 @@ public class TestRMNodeTransitions {
     Resource originalCapacity = node.getOriginalTotalCapability();
     assertEquals("Original total capability not null after recommission", null, originalCapacity);
   }
+
+  @Test
+  public void testDisappearingContainer() {
+    ContainerId cid1 = BuilderUtils.newContainerId(
+        BuilderUtils.newApplicationAttemptId(
+            BuilderUtils.newApplicationId(1, 1), 1), 1);
+    ContainerId cid2 = BuilderUtils.newContainerId(
+        BuilderUtils.newApplicationAttemptId(
+            BuilderUtils.newApplicationId(2, 2), 2), 2);
+    ArrayList<ContainerStatus> containerStats =
+        new ArrayList<ContainerStatus>();
+    containerStats.add(ContainerStatus.newInstance(cid1,
+        ContainerState.RUNNING, "", -1));
+    containerStats.add(ContainerStatus.newInstance(cid2,
+        ContainerState.RUNNING, "", -1));
+    node = getRunningNode();
+    node.handle(getMockRMNodeStatusEvent(containerStats));
+    assertEquals("unexpected number of running containers",
+        2, node.getLaunchedContainers().size());
+    Assert.assertTrue("first container not running",
+        node.getLaunchedContainers().contains(cid1));
+    Assert.assertTrue("second container not running",
+        node.getLaunchedContainers().contains(cid2));
+    assertEquals("already completed containers",
+        0, completedContainers.size());
+    containerStats.remove(0);
+    node.handle(getMockRMNodeStatusEvent(containerStats));
+    assertEquals("expected one container to be completed",
+        1, completedContainers.size());
+    ContainerStatus cs = completedContainers.get(0);
+    assertEquals("first container not the one that completed",
+        cid1, cs.getContainerId());
+    assertEquals("completed container not marked complete",
+        ContainerState.COMPLETE, cs.getState());
+    assertEquals("completed container not marked aborted",
+        ContainerExitStatus.ABORTED, cs.getExitStatus());
+    Assert.assertTrue("completed container not marked missing",
+        cs.getDiagnostics().contains("not reported"));
+    assertEquals("unexpected number of running containers",
+        1, node.getLaunchedContainers().size());
+    Assert.assertTrue("second container not running",
+        node.getLaunchedContainers().contains(cid2));
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[04/51] [abbrv] hadoop git commit: Addendum patch for YARN-5180 updating findbugs-exclude.xml

Posted by vv...@apache.org.
Addendum patch for YARN-5180 updating findbugs-exclude.xml


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

Branch: refs/heads/YARN-3926
Commit: 8554aee1bef5aff9e49e5e9119d6a7a4abf1c432
Parents: 733f3f1
Author: Arun Suresh <as...@apache.org>
Authored: Tue Jun 7 15:59:13 2016 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Tue Jun 7 15:59:43 2016 -0700

----------------------------------------------------------------------
 .../hadoop-yarn/dev-support/findbugs-exclude.xml                | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8554aee1/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 81c7e6a..6998d75 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -517,6 +517,11 @@
   </Match>
 
   <Match>
+    <Class name="org.apache.hadoop.yarn.api.records.ResourceRequest" />
+    <Method name="equals" />
+    <Bug pattern="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE" />
+  </Match>
+  <Match>
     <Package name="org.apache.hadoop.yarn.api.records.impl.pb" />
     <Bug pattern="NP_BOOLEAN_RETURN_NULL" />
   </Match>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[39/51] [abbrv] hadoop git commit: YARN-5082. Limit ContainerId increase in fair scheduler if the num of node app reserved reached the limit (sandflee via asuresh)

Posted by vv...@apache.org.
YARN-5082. Limit ContainerId increase in fair scheduler if the num of node app reserved reached the limit (sandflee via asuresh)


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

Branch: refs/heads/YARN-3926
Commit: 5279af7cd4afb090da742a96b5786d9dee6224bc
Parents: e0f4620
Author: Arun Suresh <as...@apache.org>
Authored: Fri Jun 10 22:33:42 2016 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Fri Jun 10 22:33:42 2016 -0700

----------------------------------------------------------------------
 .../scheduler/fair/FSAppAttempt.java            | 62 ++++++++++-------
 .../scheduler/fair/TestFairScheduler.java       | 72 ++++++++++++++++++++
 2 files changed, 108 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5279af7c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
index 5b83c9a..634f667 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
@@ -73,7 +73,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
       = new DefaultResourceCalculator();
 
   private long startTime;
-  private Priority priority;
+  private Priority appPriority;
   private ResourceWeights resourceWeights;
   private Resource demand = Resources.createResource(0);
   private FairScheduler scheduler;
@@ -107,7 +107,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
 
     this.scheduler = scheduler;
     this.startTime = scheduler.getClock().getTime();
-    this.priority = Priority.newInstance(1);
+    this.appPriority = Priority.newInstance(1);
     this.resourceWeights = new ResourceWeights();
   }
 
@@ -309,7 +309,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     }
 
     // default level is NODE_LOCAL
-    if (! allowedLocalityLevel.containsKey(priority)) {
+    if (!allowedLocalityLevel.containsKey(priority)) {
       // add the initial time of priority to prevent comparing with FsApp
       // startTime and allowedLocalityLevel degrade
       lastScheduledContainer.put(priority, currentTimeMs);
@@ -353,7 +353,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
 
   synchronized public RMContainer allocate(NodeType type, FSSchedulerNode node,
       Priority priority, ResourceRequest request,
-      Container container) {
+      Container reservedContainer) {
     // Update allowed locality level
     NodeType allowed = allowedLocalityLevel.get(priority);
     if (allowed != null) {
@@ -373,9 +373,15 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     if (getTotalRequiredResources(priority) <= 0) {
       return null;
     }
+
+    Container container = reservedContainer;
+    if (container == null) {
+      container =
+          createContainer(node, request.getCapability(), request.getPriority());
+    }
     
     // Create RMContainer
-    RMContainer rmContainer = new RMContainerImpl(container, 
+    RMContainer rmContainer = new RMContainerImpl(container,
         getApplicationAttemptId(), node.getNodeID(),
         appSchedulingInfo.getUser(), rmContext);
     ((RMContainerImpl)rmContainer).setQueueName(this.getQueueName());
@@ -485,21 +491,26 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
    * in {@link FSSchedulerNode}..
    * return whether reservation was possible with the current threshold limits
    */
-  private boolean reserve(Priority priority, FSSchedulerNode node,
-      Container container, NodeType type, boolean alreadyReserved) {
+  private boolean reserve(ResourceRequest request, FSSchedulerNode node,
+      Container reservedContainer, NodeType type) {
 
+    Priority priority = request.getPriority();
     if (!reservationExceedsThreshold(node, type)) {
       LOG.info("Making reservation: node=" + node.getNodeName() +
               " app_id=" + getApplicationId());
-      if (!alreadyReserved) {
-        getMetrics().reserveResource(getUser(), container.getResource());
+      if (reservedContainer == null) {
+        reservedContainer =
+            createContainer(node, request.getCapability(),
+              request.getPriority());
+        getMetrics().reserveResource(getUser(),
+            reservedContainer.getResource());
         RMContainer rmContainer =
-                super.reserve(node, priority, null, container);
+                super.reserve(node, priority, null, reservedContainer);
         node.reserveResource(this, priority, rmContainer);
         setReservation(node);
       } else {
         RMContainer rmContainer = node.getReservedContainer();
-        super.reserve(node, priority, rmContainer, container);
+        super.reserve(node, priority, rmContainer, reservedContainer);
         node.reserveResource(this, priority, rmContainer);
         setReservation(node);
       }
@@ -615,18 +626,17 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     // How much does the node have?
     Resource available = node.getUnallocatedResource();
 
-    Container container = null;
+    Container reservedContainer = null;
     if (reserved) {
-      container = node.getReservedContainer().getContainer();
-    } else {
-      container = createContainer(node, capability, request.getPriority());
+      reservedContainer = node.getReservedContainer().getContainer();
     }
 
     // Can we allocate a container on this node?
     if (Resources.fitsIn(capability, available)) {
       // Inform the application of the new container for this request
       RMContainer allocatedContainer =
-          allocate(type, node, request.getPriority(), request, container);
+          allocate(type, node, request.getPriority(), request,
+              reservedContainer);
       if (allocatedContainer == null) {
         // Did the application need this resource?
         if (reserved) {
@@ -647,30 +657,30 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
       // the AM. Set the amResource for this app and update the leaf queue's AM
       // usage
       if (!isAmRunning() && !getUnmanagedAM()) {
-        setAMResource(container.getResource());
-        getQueue().addAMResourceUsage(container.getResource());
+        setAMResource(capability);
+        getQueue().addAMResourceUsage(capability);
         setAmRunning(true);
       }
 
-      return container.getResource();
+      return capability;
     }
 
     // The desired container won't fit here, so reserve
-    if (isReservable(container) &&
-        reserve(request.getPriority(), node, container, type, reserved)) {
+    if (isReservable(capability) &&
+        reserve(request, node, reservedContainer, type)) {
       return FairScheduler.CONTAINER_RESERVED;
     } else {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Not creating reservation as container " + container.getId()
-            + " is not reservable");
+        LOG.debug("Couldn't creating reservation for " +
+            getName() + ",at priority " +  request.getPriority());
       }
       return Resources.none();
     }
   }
 
-  private boolean isReservable(Container container) {
+  private boolean isReservable(Resource capacity) {
     return scheduler.isAtLeastReservationThreshold(
-      getQueue().getPolicy().getResourceCalculator(), container.getResource());
+        getQueue().getPolicy().getResourceCalculator(), capacity);
   }
 
   private boolean hasNodeOrRackLocalRequests(Priority priority) {
@@ -907,7 +917,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
   public Priority getPriority() {
     // Right now per-app priorities are not passed to scheduler,
     // so everyone has the same priority.
-    return priority;
+    return appPriority;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5279af7c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index 3e5a40f..ec77a9b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -4480,4 +4480,76 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     resourceManager.getResourceScheduler().handle(nodeAddEvent1);
     return nm;
   }
+
+  @Test(timeout = 120000)
+  public void testContainerAllocationWithContainerIdLeap() throws Exception {
+    conf.setFloat(FairSchedulerConfiguration.RESERVABLE_NODES, 0.50f);
+    scheduler.init(conf);
+    scheduler.start();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
+    // Add two node
+    RMNode node1 = MockNodes.newNodeInfo(1,
+        Resources.createResource(3072, 10), 1, "127.0.0.1");
+    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
+    scheduler.handle(nodeEvent1);
+
+    RMNode node2 = MockNodes.newNodeInfo(1,
+        Resources.createResource(3072, 10), 1, "127.0.0.2");
+    NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
+    scheduler.handle(nodeEvent2);
+
+    ApplicationAttemptId app1 =
+        createSchedulingRequest(2048, "queue1", "user1", 2);
+    scheduler.update();
+    scheduler.handle(new NodeUpdateSchedulerEvent(node1));
+    scheduler.handle(new NodeUpdateSchedulerEvent(node2));
+
+    ApplicationAttemptId app2 =
+        createSchedulingRequest(2048, "queue1", "user1", 1);
+    scheduler.update();
+    scheduler.handle(new NodeUpdateSchedulerEvent(node1));
+    scheduler.handle(new NodeUpdateSchedulerEvent(node2));
+
+    assertEquals(4096, scheduler.getQueueManager().getQueue("queue1").
+        getResourceUsage().getMemory());
+
+    //container will be reserved at node1
+    RMContainer reservedContainer1 =
+        scheduler.getSchedulerNode(node1.getNodeID()).getReservedContainer();
+    assertNotEquals(reservedContainer1, null);
+    RMContainer reservedContainer2 =
+        scheduler.getSchedulerNode(node2.getNodeID()).getReservedContainer();
+    assertEquals(reservedContainer2, null);
+
+    for (int i = 0; i < 10; i++) {
+      scheduler.handle(new NodeUpdateSchedulerEvent(node1));
+      scheduler.handle(new NodeUpdateSchedulerEvent(node2));
+    }
+
+    // release resource
+    scheduler.handle(new AppAttemptRemovedSchedulerEvent(
+        app1, RMAppAttemptState.KILLED, false));
+
+    assertEquals(0, scheduler.getQueueManager().getQueue("queue1").
+        getResourceUsage().getMemory());
+
+    // container will be allocated at node2
+    scheduler.handle(new NodeUpdateSchedulerEvent(node2));
+    assertEquals(scheduler.getSchedulerApp(app2).
+        getLiveContainers().size(), 1);
+
+    long maxId = 0;
+    for (RMContainer container :
+        scheduler.getSchedulerApp(app2).getLiveContainers()) {
+      assertTrue(
+          container.getContainer().getNodeId().equals(node2.getNodeID()));
+      if (container.getContainerId().getContainerId() > maxId) {
+        maxId = container.getContainerId().getContainerId();
+      }
+    }
+
+    long reservedId = reservedContainer1.getContainerId().getContainerId();
+    assertEquals(reservedId + 1, maxId);
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[30/51] [abbrv] hadoop git commit: YARN-3426. Add jdiff support to YARN. (vinodkv via wangda)

Posted by vv...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/03fc6b1b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.6.0.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.6.0.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.6.0.xml
new file mode 100644
index 0000000..9e37e8a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.6.0.xml
@@ -0,0 +1,2870 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!--
+   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.
+-->
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Wed Apr 08 11:30:04 PDT 2015 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="hadoop-yarn-common 2.6.0"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.ExcludePrivateAnnotationsJDiffDoclet -docletpath /Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/hadoop-annotations.jar:/Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/jdiff.jar -verbose -classpath /Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/classes:/Users/llu/.m2/repository/org/apache/hadoop/hadoop-common/2.6.0/hadoop-common-2.6.0.jar:/Users/llu/.m2/repository/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/Users/llu/.m2/repository/xmlenc/xmlenc/0.52/xmlenc-0.52.jar:/Users/llu/.m2/repository/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1.jar:/Users/llu/.m2/repository/commons-net/commons-net/3.1/commons-net-3.1.jar:/Users/llu/.m2/repository/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1.jar:/Users/llu/.m2/repository/org/mortbay/jetty/jetty/6.1.26/jetty
 -6.1.26.jar:/Users/llu/.m2/repository/javax/servlet/jsp/jsp-api/2.1/jsp-api-2.1.jar:/Users/llu/.m2/repository/net/java/dev/jets3t/jets3t/0.9.0/jets3t-0.9.0.jar:/Users/llu/.m2/repository/org/apache/httpcomponents/httpclient/4.2.5/httpclient-4.2.5.jar:/Users/llu/.m2/repository/org/apache/httpcomponents/httpcore/4.2.5/httpcore-4.2.5.jar:/Users/llu/.m2/repository/com/jamesmurty/utils/java-xmlbuilder/0.4/java-xmlbuilder-0.4.jar:/Users/llu/.m2/repository/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/Users/llu/.m2/repository/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/Users/llu/.m2/repository/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/Users/llu/.m2/repository/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar:/Users/llu/.m2/repository/org/slf4j/slf4j-log4j12/1.7.5/slf4j-log4j12-1.7.5.jar:/Users/llu/.m2/repository/org/apache/avro/avro/1.7.4/avro-1.7.4.jar:/Users/llu/.m2/repository/c
 om/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/Users/llu/.m2/repository/org/xerial/snappy/snappy-java/1.0.4.1/snappy-java-1.0.4.1.jar:/Users/llu/.m2/repository/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/Users/llu/.m2/repository/org/apache/hadoop/hadoop-auth/2.6.0/hadoop-auth-2.6.0.jar:/Users/llu/.m2/repository/org/apache/directory/server/apacheds-kerberos-codec/2.0.0-M15/apacheds-kerberos-codec-2.0.0-M15.jar:/Users/llu/.m2/repository/org/apache/directory/server/apacheds-i18n/2.0.0-M15/apacheds-i18n-2.0.0-M15.jar:/Users/llu/.m2/repository/org/apache/directory/api/api-asn1-api/1.0.0-M20/api-asn1-api-1.0.0-M20.jar:/Users/llu/.m2/repository/org/apache/directory/api/api-util/1.0.0-M20/api-util-1.0.0-M20.jar:/Users/llu/.m2/repository/org/apache/curator/curator-framework/2.6.0/curator-framework-2.6.0.jar:/Users/llu/.m2/repository/com/jcraft/jsch/0.1.42/jsch-0.1.42.jar:/Users/llu/.m2/repository/org/apache/curator/curator-client/2.6.0/curator-client-2.6.0.jar:/Users/llu/.m2/
 repository/org/apache/curator/curator-recipes/2.6.0/curator-recipes-2.6.0.jar:/Users/llu/.m2/repository/com/google/code/findbugs/jsr305/1.3.9/jsr305-1.3.9.jar:/Users/llu/.m2/repository/org/htrace/htrace-core/3.0.4/htrace-core-3.0.4.jar:/Users/llu/.m2/repository/org/apache/zookeeper/zookeeper/3.4.6/zookeeper-3.4.6.jar:/Users/llu/.m2/repository/io/netty/netty/3.6.2.Final/netty-3.6.2.Final.jar:/Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/hadoop-yarn-api-2.6.0.jar:/Users/llu/.m2/repository/javax/xml/bind/jaxb-api/2.2.2/jaxb-api-2.2.2.jar:/Users/llu/.m2/repository/javax/xml/stream/stax-api/1.0-2/stax-api-1.0-2.jar:/Users/llu/.m2/repository/javax/activation/activation/1.1/activation-1.1.jar:/Users/llu/.m2/repository/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/Users/llu/.m2/repository/org/tukaani/xz/1.0/xz-1.0.jar:/Users/llu/.m2/repository/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/Users/llu/.m2/repository/javax/servlet/serv
 let-api/2.5/servlet-api-2.5.jar:/Users/llu/.m2/repository/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/Users/llu/.m2/repository/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/Users/llu/.m2/repository/com/sun/jersey/jersey-core/1.9/jersey-core-1.9.jar:/Users/llu/.m2/repository/com/sun/jersey/jersey-client/1.9/jersey-client-1.9.jar:/Users/llu/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/llu/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/llu/.m2/repository/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/Users/llu/.m2/repository/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/Users/llu/.m2/repository/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/Users/llu/.m2/repository/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/Users/llu/.m2/repository/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/Users/llu/.m2/repositor
 y/org/slf4j/slf4j-api/1.7.5/slf4j-api-1.7.5.jar:/Users/llu/.m2/repository/org/apache/hadoop/hadoop-annotations/2.6.0/hadoop-annotations-2.6.0.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/lib/tools.jar:/Users/llu/.m2/repository/com/google/inject/extensions/guice-servlet/3.0/guice-servlet-3.0.jar:/Users/llu/.m2/repository/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar:/Users/llu/.m2/repository/commons-io/commons-io/2.4/commons-io-2.4.jar:/Users/llu/.m2/repository/com/google/inject/guice/3.0/guice-3.0.jar:/Users/llu/.m2/repository/javax/inject/javax.inject/1/javax.inject-1.jar:/Users/llu/.m2/repository/aopalliance/aopalliance/1.0/aopalliance-1.0.jar:/Users/llu/.m2/repository/com/sun/jersey/jersey-server/1.9/jersey-server-1.9.jar:/Users/llu/.m2/repository/asm/asm/3.2/asm-3.2.jar:/Users/llu/.m2/repository/com/sun/jersey/jersey-json/1.9/jersey-json-1.9.jar:/Users/llu/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/Users/llu/.m2/r
 epository/com/sun/xml/bind/jaxb-impl/2.2.3-1/jaxb-impl-2.2.3-1.jar:/Users/llu/.m2/repository/com/sun/jersey/contribs/jersey-guice/1.9/jersey-guice-1.9.jar:/Users/llu/.m2/repository/log4j/log4j/1.2.17/log4j-1.2.17.jar -sourcepath /Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java -apidir /Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/site/jdiff/xml -apiname hadoop-yarn-common 2.6.0 -->
+<package name="org.apache.hadoop.yarn">
+  <!-- start class org.apache.hadoop.yarn.ContainerLogAppender -->
+  <class name="ContainerLogAppender" extends="org.apache.log4j.FileAppender"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.io.Flushable"/>
+    <constructor name="ContainerLogAppender"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="activateOptions"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="append"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="event" type="org.apache.log4j.spi.LoggingEvent"/>
+    </method>
+    <method name="flush"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="close"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getContainerLogDir" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Getter/Setter methods for log4j.]]>
+      </doc>
+    </method>
+    <method name="setContainerLogDir"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerLogDir" type="java.lang.String"/>
+    </method>
+    <method name="getTotalLogFileSize" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setTotalLogFileSize"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="logSize" type="long"/>
+    </method>
+    <doc>
+    <![CDATA[A simple log4j-appender for container's logs.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.ContainerLogAppender -->
+  <!-- start class org.apache.hadoop.yarn.ContainerRollingLogAppender -->
+  <class name="ContainerRollingLogAppender" extends="org.apache.log4j.RollingFileAppender"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.io.Flushable"/>
+    <constructor name="ContainerRollingLogAppender"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="activateOptions"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="flush"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getContainerLogDir" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Getter/Setter methods for log4j.]]>
+      </doc>
+    </method>
+    <method name="setContainerLogDir"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerLogDir" type="java.lang.String"/>
+    </method>
+    <doc>
+    <![CDATA[A simple log4j-appender for container's logs.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.ContainerRollingLogAppender -->
+  <!-- start class org.apache.hadoop.yarn.YarnUncaughtExceptionHandler -->
+  <class name="YarnUncaughtExceptionHandler" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.lang.Thread.UncaughtExceptionHandler"/>
+    <constructor name="YarnUncaughtExceptionHandler"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="uncaughtException"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="t" type="java.lang.Thread"/>
+      <param name="e" type="java.lang.Throwable"/>
+    </method>
+    <doc>
+    <![CDATA[This class is intended to be installed by calling
+ {@link Thread#setDefaultUncaughtExceptionHandler(UncaughtExceptionHandler)}
+ In the main entry point.  It is intended to try and cleanly shut down
+ programs using the Yarn Event framework.
+
+ Note: Right now it only will shut down the program if a Error is caught, but
+ not any other exception.  Anything else is just logged.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.YarnUncaughtExceptionHandler -->
+</package>
+<package name="org.apache.hadoop.yarn.api">
+</package>
+<package name="org.apache.hadoop.yarn.client">
+  <!-- start class org.apache.hadoop.yarn.client.AHSProxy -->
+  <class name="AHSProxy" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AHSProxy"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createAHSProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="ahsAddress" type="java.net.InetSocketAddress"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="rmAddress" type="java.net.InetSocketAddress"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.AHSProxy -->
+  <!-- start class org.apache.hadoop.yarn.client.ClientRMProxy -->
+  <class name="ClientRMProxy" extends="org.apache.hadoop.yarn.client.RMProxy"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="createRMProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="configuration" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Create a proxy to the ResourceManager for the specified protocol.
+ @param configuration Configuration with all the required information.
+ @param protocol Client protocol for which proxy is being requested.
+ @param <T> Type of proxy.
+ @return Proxy to the ResourceManager for the specified client protocol.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getRMDelegationTokenService" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <doc>
+      <![CDATA[Get the token service name to be used for RMDelegationToken. Depending
+ on whether HA is enabled or not, this method generates the appropriate
+ service name as a comma-separated list of service addresses.
+
+ @param conf Configuration corresponding to the cluster we need the
+             RMDelegationToken for
+ @return - Service name for RMDelegationToken]]>
+      </doc>
+    </method>
+    <method name="getAMRMTokenService" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getTokenService" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="address" type="java.lang.String"/>
+      <param name="defaultAddr" type="java.lang.String"/>
+      <param name="defaultPort" type="int"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.ClientRMProxy -->
+  <!-- start class org.apache.hadoop.yarn.client.NMProxy -->
+  <class name="NMProxy" extends="org.apache.hadoop.yarn.client.ServerProxy"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NMProxy"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createNMProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="ugi" type="org.apache.hadoop.security.UserGroupInformation"/>
+      <param name="rpc" type="org.apache.hadoop.yarn.ipc.YarnRPC"/>
+      <param name="serverAddress" type="java.net.InetSocketAddress"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.NMProxy -->
+  <!-- start class org.apache.hadoop.yarn.client.RMHAServiceTarget -->
+  <class name="RMHAServiceTarget" extends="org.apache.hadoop.ha.HAServiceTarget"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="RMHAServiceTarget" type="org.apache.hadoop.yarn.conf.YarnConfiguration"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </constructor>
+    <method name="getAddress" return="java.net.InetSocketAddress"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getZKFCAddress" return="java.net.InetSocketAddress"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getFencer" return="org.apache.hadoop.ha.NodeFencer"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="checkFencingConfigured"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="BadFencingConfigurationException" type="org.apache.hadoop.ha.BadFencingConfigurationException"/>
+    </method>
+    <method name="isAutoFailoverEnabled" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.RMHAServiceTarget -->
+  <!-- start class org.apache.hadoop.yarn.client.RMProxy -->
+  <class name="RMProxy" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="RMProxy"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createRMProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="This method is deprecated and is not used by YARN internally any more.
+ To create a proxy to the RM, use ClientRMProxy#createRMProxy or
+ ServerRMProxy#createRMProxy.
+
+ Create a proxy to the ResourceManager at the specified address.">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="rmAddress" type="java.net.InetSocketAddress"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[@deprecated
+ This method is deprecated and is not used by YARN internally any more.
+ To create a proxy to the RM, use ClientRMProxy#createRMProxy or
+ ServerRMProxy#createRMProxy.
+
+ Create a proxy to the ResourceManager at the specified address.
+
+ @param conf Configuration to generate retry policy
+ @param protocol Protocol for the proxy
+ @param rmAddress Address of the ResourceManager
+ @param <T> Type information of the proxy
+ @return Proxy to the RM
+ @throws IOException]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.RMProxy -->
+  <!-- start class org.apache.hadoop.yarn.client.ServerProxy -->
+  <class name="ServerProxy" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ServerProxy"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createRetryPolicy" return="org.apache.hadoop.io.retry.RetryPolicy"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="maxWaitTimeStr" type="java.lang.String"/>
+      <param name="defMaxWaitTime" type="long"/>
+      <param name="connectRetryIntervalStr" type="java.lang.String"/>
+      <param name="defRetryInterval" type="long"/>
+    </method>
+    <method name="createRetriableProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="user" type="org.apache.hadoop.security.UserGroupInformation"/>
+      <param name="rpc" type="org.apache.hadoop.yarn.ipc.YarnRPC"/>
+      <param name="serverAddress" type="java.net.InetSocketAddress"/>
+      <param name="retryPolicy" type="org.apache.hadoop.io.retry.RetryPolicy"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.ServerProxy -->
+</package>
+<package name="org.apache.hadoop.yarn.client.api">
+  <!-- start class org.apache.hadoop.yarn.client.api.TimelineClient -->
+  <class name="TimelineClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="TimelineClient" type="java.lang.String"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createTimelineClient" return="org.apache.hadoop.yarn.client.api.TimelineClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="putEntities" return="org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="entities" type="org.apache.hadoop.yarn.api.records.timeline.TimelineEntity[]"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Send the information of a number of conceptual entities to the timeline
+ server. It is a blocking API. The method will not return until it gets the
+ response from the timeline server.
+ </p>
+
+ @param entities
+          the collection of {@link TimelineEntity}
+ @return the error information if the sent entities are not correctly stored
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="putDomain"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="domain" type="org.apache.hadoop.yarn.api.records.timeline.TimelineDomain"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Send the information of a domain to the timeline server. It is a
+ blocking API. The method will not return until it gets the response from
+ the timeline server.
+ </p>
+
+ @param domain
+          an {@link TimelineDomain} object
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="getDelegationToken" return="org.apache.hadoop.security.token.Token"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="renewer" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a delegation token so as to be able to talk to the timeline server in a
+ secure way.
+ </p>
+
+ @param renewer
+          Address of the renewer who can renew these tokens when needed by
+          securely talking to the timeline server
+ @return a delegation token ({@link Token}) that can be used to talk to the
+         timeline server
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="renewDelegationToken" return="long"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="timelineDT" type="org.apache.hadoop.security.token.Token"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Renew a timeline delegation token.
+ </p>
+
+ @param timelineDT
+          the delegation token to renew
+ @return the new expiration time
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="cancelDelegationToken"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="timelineDT" type="org.apache.hadoop.security.token.Token"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Cancel a timeline delegation token.
+ </p>
+
+ @param timelineDT
+          the delegation token to cancel
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[A client library that can be used to post some information in terms of a
+ number of conceptual entities.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.TimelineClient -->
+</package>
+<package name="org.apache.hadoop.yarn.client.api.impl">
+</package>
+<package name="org.apache.hadoop.yarn.event">
+  <!-- start class org.apache.hadoop.yarn.event.AbstractEvent -->
+  <class name="AbstractEvent" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.event.Event"/>
+    <constructor name="AbstractEvent" type="TYPE"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AbstractEvent" type="TYPE, long"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getTimestamp" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getType" return="TYPE"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <doc>
+    <![CDATA[Parent class of all the events. All events extend this class.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.event.AbstractEvent -->
+  <!-- start class org.apache.hadoop.yarn.event.AsyncDispatcher -->
+  <class name="AsyncDispatcher" extends="org.apache.hadoop.service.AbstractService"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.event.Dispatcher"/>
+    <constructor name="AsyncDispatcher"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AsyncDispatcher" type="java.util.concurrent.BlockingQueue"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="serviceInit"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="serviceStart"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="setDrainEventsOnStop"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="serviceStop"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="dispatch"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="event" type="org.apache.hadoop.yarn.event.Event"/>
+    </method>
+    <method name="register"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="eventType" type="java.lang.Class"/>
+      <param name="handler" type="org.apache.hadoop.yarn.event.EventHandler"/>
+    </method>
+    <method name="getEventHandler" return="org.apache.hadoop.yarn.event.EventHandler"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="eventDispatchers" type="java.util.Map"
+      transient="false" volatile="false"
+      static="false" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[Dispatches {@link Event}s in a separate thread. Currently only single thread
+ does that. Potentially there could be multiple channels for each event type
+ class and a thread pool can be used to dispatch the events.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.event.AsyncDispatcher -->
+  <!-- start interface org.apache.hadoop.yarn.event.Dispatcher -->
+  <interface name="Dispatcher"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getEventHandler" return="org.apache.hadoop.yarn.event.EventHandler"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="register"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="eventType" type="java.lang.Class"/>
+      <param name="handler" type="org.apache.hadoop.yarn.event.EventHandler"/>
+    </method>
+    <field name="DISPATCHER_EXIT_ON_ERROR_KEY" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="DEFAULT_DISPATCHER_EXIT_ON_ERROR" type="boolean"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[Event Dispatcher interface. It dispatches events to registered
+ event handlers based on event types.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.event.Dispatcher -->
+  <!-- start interface org.apache.hadoop.yarn.event.Event -->
+  <interface name="Event"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getType" return="TYPE"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getTimestamp" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <doc>
+    <![CDATA[Interface defining events api.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.event.Event -->
+  <!-- start interface org.apache.hadoop.yarn.event.EventHandler -->
+  <interface name="EventHandler"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="handle"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="event" type="T"/>
+    </method>
+    <doc>
+    <![CDATA[Interface for handling events of type T
+
+ @param <T> parameterized event of type T]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.event.EventHandler -->
+</package>
+<package name="org.apache.hadoop.yarn.logaggregation">
+  <!-- start class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat -->
+  <class name="AggregatedLogFormat" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AggregatedLogFormat"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat -->
+  <!-- start class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey -->
+  <class name="AggregatedLogFormat.LogKey" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.io.Writable"/>
+    <constructor name="AggregatedLogFormat.LogKey"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AggregatedLogFormat.LogKey" type="org.apache.hadoop.yarn.api.records.ContainerId"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AggregatedLogFormat.LogKey" type="java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="obj" type="java.lang.Object"/>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey -->
+  <!-- start class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogReader -->
+  <class name="AggregatedLogFormat.LogReader" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AggregatedLogFormat.LogReader" type="org.apache.hadoop.conf.Configuration, org.apache.hadoop.fs.Path"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </constructor>
+    <method name="getApplicationOwner" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Returns the owner of the application.
+
+ @return the application owner.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAcls" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Returns ACLs for the application. An empty map is returned if no ACLs are
+ found.
+
+ @return a map of the Application ACLs.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="next" return="java.io.DataInputStream"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="key" type="org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Read the next key and return the value-stream.
+
+ @param key
+ @return the valueStream if there are more keys or null otherwise.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="readAcontainerLogs"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="writer" type="java.io.Writer"/>
+      <param name="logUploadedTime" type="long"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Writes all logs for a single container to the provided writer.
+ @param valueStream
+ @param writer
+ @param logUploadedTime
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="readAcontainerLogs"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="writer" type="java.io.Writer"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Writes all logs for a single container to the provided writer.
+ @param valueStream
+ @param writer
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="readAContainerLogsForALogType"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="out" type="java.io.PrintStream"/>
+      <param name="logUploadedTime" type="long"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Keep calling this till you get a {@link EOFException} for getting logs of
+ all types for a single container.
+
+ @param valueStream
+ @param out
+ @param logUploadedTime
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="readAContainerLogsForALogType"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="out" type="java.io.PrintStream"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Keep calling this till you get a {@link EOFException} for getting logs of
+ all types for a single container.
+
+ @param valueStream
+ @param out
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="close"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogReader -->
+  <!-- start class org.apache.hadoop.yarn.logaggregation.LogCLIHelpers -->
+  <class name="LogCLIHelpers" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.conf.Configurable"/>
+    <constructor name="LogCLIHelpers"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="setConf"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getConf" return="org.apache.hadoop.conf.Configuration"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.logaggregation.LogCLIHelpers -->
+</package>
+<package name="org.apache.hadoop.yarn.nodelabels">
+  <!-- start class org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager -->
+  <class name="CommonNodeLabelsManager" extends="org.apache.hadoop.service.AbstractService"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="CommonNodeLabelsManager"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="handleStoreEvent"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="event" type="org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEvent"/>
+    </method>
+    <method name="initDispatcher"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="serviceInit"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="initNodeLabelStore"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="startDispatcher"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </method>
+    <method name="serviceStart"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="stopDispatcher"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </method>
+    <method name="serviceStop"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="addToCluserNodeLabels"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="labels" type="java.util.Set"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Add multiple node labels to repository
+
+ @param labels
+          new node labels added]]>
+      </doc>
+    </method>
+    <method name="checkAddLabelsToNode"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="addedLabelsToNode" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="internalAddLabelsToNode"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="addedLabelsToNode" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="addLabelsToNode"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="addedLabelsToNode" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[add more labels to nodes
+
+ @param addedLabelsToNode node -> labels map]]>
+      </doc>
+    </method>
+    <method name="checkRemoveFromClusterNodeLabels"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="labelsToRemove" type="java.util.Collection"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="internalRemoveFromClusterNodeLabels"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="labelsToRemove" type="java.util.Collection"/>
+    </method>
+    <method name="removeFromClusterNodeLabels"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="labelsToRemove" type="java.util.Collection"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Remove multiple node labels from repository
+
+ @param labelsToRemove
+          node labels to remove
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="checkRemoveLabelsFromNode"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="removeLabelsFromNode" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="internalRemoveLabelsFromNode"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="removeLabelsFromNode" type="java.util.Map"/>
+    </method>
+    <method name="removeLabelsFromNode"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="removeLabelsFromNode" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[remove labels from nodes, labels being removed most be contained by these
+ nodes
+
+ @param removeLabelsFromNode node -> labels map]]>
+      </doc>
+    </method>
+    <method name="checkReplaceLabelsOnNode"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="replaceLabelsToNode" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="internalReplaceLabelsOnNode"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="replaceLabelsToNode" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="replaceLabelsOnNode"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="replaceLabelsToNode" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[replace labels to nodes
+
+ @param replaceLabelsToNode node -> labels map]]>
+      </doc>
+    </method>
+    <method name="getNodeLabels" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get mapping of nodes to labels
+
+ @return nodes to labels map]]>
+      </doc>
+    </method>
+    <method name="getClusterNodeLabels" return="java.util.Set"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get existing valid labels in repository
+
+ @return existing valid labels in repository]]>
+      </doc>
+    </method>
+    <method name="normalizeLabel" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="label" type="java.lang.String"/>
+    </method>
+    <method name="getNMInNodeSet" return="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Node"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+    </method>
+    <method name="getNMInNodeSet" return="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Node"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <param name="map" type="java.util.Map"/>
+    </method>
+    <method name="getNMInNodeSet" return="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Node"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <param name="map" type="java.util.Map"/>
+      <param name="checkRunning" type="boolean"/>
+    </method>
+    <method name="getLabelsByNode" return="java.util.Set"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+    </method>
+    <method name="getLabelsByNode" return="java.util.Set"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <param name="map" type="java.util.Map"/>
+    </method>
+    <method name="createNodeIfNonExisted"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="createHostIfNonExisted"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="hostName" type="java.lang.String"/>
+    </method>
+    <method name="normalizeNodeIdToLabels" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="nodeIdToLabels" type="java.util.Map"/>
+    </method>
+    <field name="LOG" type="org.apache.commons.logging.Log"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="EMPTY_STRING_SET" type="java.util.Set"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="ANY" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="ACCESS_ANY_LABEL_SET" type="java.util.Set"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="WILDCARD_PORT" type="int"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="NO_LABEL" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[If a user doesn't specify label of a queue or node, it belongs
+ DEFAULT_LABEL]]>
+      </doc>
+    </field>
+    <field name="dispatcher" type="org.apache.hadoop.yarn.event.Dispatcher"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="labelCollections" type="java.util.concurrent.ConcurrentMap"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="nodeCollections" type="java.util.concurrent.ConcurrentMap"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="readLock" type="java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock"
+      transient="false" volatile="false"
+      static="false" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="writeLock" type="java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock"
+      transient="false" volatile="false"
+      static="false" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="store" type="org.apache.hadoop.yarn.nodelabels.NodeLabelsStore"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Host -->
+  <class name="CommonNodeLabelsManager.Host" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="protected"
+    deprecated="not deprecated">
+    <constructor name="CommonNodeLabelsManager.Host"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="copy" return="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Host"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="labels" type="java.util.Set"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="nms" type="java.util.Map"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[A <code>Host</code> can have multiple <code>Node</code>s]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Host -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Label -->
+  <class name="CommonNodeLabelsManager.Label" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="protected"
+    deprecated="not deprecated">
+    <constructor name="CommonNodeLabelsManager.Label"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <field name="resource" type="org.apache.hadoop.yarn.api.records.Resource"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Label -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Node -->
+  <class name="CommonNodeLabelsManager.Node" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="protected"
+    deprecated="not deprecated">
+    <constructor name="CommonNodeLabelsManager.Node"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="copy" return="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Node"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="labels" type="java.util.Set"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="resource" type="org.apache.hadoop.yarn.api.records.Resource"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="running" type="boolean"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Node -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore -->
+  <class name="FileSystemNodeLabelsStore" extends="org.apache.hadoop.yarn.nodelabels.NodeLabelsStore"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="FileSystemNodeLabelsStore" type="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="init"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="close"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="updateNodeToLabelsMappings"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeToLabels" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="storeNewClusterNodeLabels"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="labels" type="java.util.Set"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="removeClusterNodeLabels"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="labels" type="java.util.Collection"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="recover"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <field name="LOG" type="org.apache.commons.logging.Log"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="DEFAULT_DIR_NAME" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="MIRROR_FILENAME" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="EDITLOG_FILENAME" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore.SerializedLogType -->
+  <class name="FileSystemNodeLabelsStore.SerializedLogType" extends="java.lang.Enum"
+    abstract="false"
+    static="true" final="true" visibility="protected"
+    deprecated="not deprecated">
+    <method name="values" return="org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore.SerializedLogType[]"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="valueOf" return="org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore.SerializedLogType"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore.SerializedLogType -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.NodeLabelsStore -->
+  <class name="NodeLabelsStore" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.io.Closeable"/>
+    <constructor name="NodeLabelsStore" type="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="updateNodeToLabelsMappings"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeToLabels" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Store node -> label]]>
+      </doc>
+    </method>
+    <method name="storeNewClusterNodeLabels"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="label" type="java.util.Set"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Store new labels]]>
+      </doc>
+    </method>
+    <method name="removeClusterNodeLabels"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="labels" type="java.util.Collection"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Remove labels]]>
+      </doc>
+    </method>
+    <method name="recover"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Recover labels and node to labels mappings from store
+ @param conf]]>
+      </doc>
+    </method>
+    <method name="init"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="getNodeLabelsManager" return="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="mgr" type="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager"
+      transient="false" volatile="false"
+      static="false" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="conf" type="org.apache.hadoop.conf.Configuration"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.NodeLabelsStore -->
+</package>
+<package name="org.apache.hadoop.yarn.nodelabels.event">
+  <!-- start class org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEvent -->
+  <class name="NodeLabelsStoreEvent" extends="org.apache.hadoop.yarn.event.AbstractEvent"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NodeLabelsStoreEvent" type="org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEventType"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEvent -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEventType -->
+  <class name="NodeLabelsStoreEventType" extends="java.lang.Enum"
+    abstract="false"
+    static="false" final="true" visibility="public"
+    deprecated="not deprecated">
+    <method name="values" return="org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEventType[]"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="valueOf" return="org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEventType"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEventType -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.event.RemoveClusterNodeLabels -->
+  <class name="RemoveClusterNodeLabels" extends="org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEvent"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="RemoveClusterNodeLabels" type="java.util.Collection"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getLabels" return="java.util.Collection"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.event.RemoveClusterNodeLabels -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.event.StoreNewClusterNodeLabels -->
+  <class name="StoreNewClusterNodeLabels" extends="org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEvent"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="StoreNewClusterNodeLabels" type="java.util.Set"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getLabels" return="java.util.Set"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.event.StoreNewClusterNodeLabels -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.event.UpdateNodeToLabelsMappingsEvent -->
+  <class name="UpdateNodeToLabelsMappingsEvent" extends="org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEvent"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="UpdateNodeToLabelsMappingsEvent" type="java.util.Map"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getNodeToLabels" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.event.UpdateNodeToLabelsMappingsEvent -->
+</package>
+<package name="org.apache.hadoop.yarn.security">
+  <!-- start class org.apache.hadoop.yarn.security.AMRMTokenIdentifier -->
+  <class name="AMRMTokenIdentifier" extends="org.apache.hadoop.security.token.TokenIdentifier"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AMRMTokenIdentifier"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AMRMTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId, int"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="write"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="out" type="java.io.DataOutput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="readFields"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.DataInput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getKind" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getUser" return="org.apache.hadoop.security.UserGroupInformation"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getKeyId" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos.AMRMTokenIdentifierProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="other" type="java.lang.Object"/>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="KIND_NAME" type="org.apache.hadoop.io.Text"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[AMRMTokenIdentifier is the TokenIdentifier to be used by
+ ApplicationMasters to authenticate to the ResourceManager.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.AMRMTokenIdentifier -->
+  <!-- start class org.apache.hadoop.yarn.security.AMRMTokenSelector -->
+  <class name="AMRMTokenSelector" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.security.token.TokenSelector"/>
+    <constructor name="AMRMTokenSelector"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="selectToken" return="org.apache.hadoop.security.token.Token"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="service" type="org.apache.hadoop.io.Text"/>
+      <param name="tokens" type="java.util.Collection"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.AMRMTokenSelector -->
+  <!-- start class org.apache.hadoop.yarn.security.ContainerManagerSecurityInfo -->
+  <class name="ContainerManagerSecurityInfo" extends="org.apache.hadoop.security.SecurityInfo"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ContainerManagerSecurityInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getKerberosInfo" return="org.apache.hadoop.security.KerberosInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getTokenInfo" return="org.apache.hadoop.security.token.TokenInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.ContainerManagerSecurityInfo -->
+  <!-- start class org.apache.hadoop.yarn.security.ContainerTokenIdentifier -->
+  <class name="ContainerTokenIdentifier" extends="org.apache.hadoop.security.token.TokenIdentifier"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ContainerTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ContainerId, java.lang.String, java.lang.String, org.apache.hadoop.yarn.api.records.Resource, long, int, long, org.apache.hadoop.yarn.api.records.Priority, long"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="ContainerTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ContainerId, java.lang.String, java.lang.String, org.apache.hadoop.yarn.api.records.Resource, long, int, long, org.apache.hadoop.yarn.api.records.Priority, long, org.apache.hadoop.yarn.api.records.LogAggregationContext"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="ContainerTokenIdentifier"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Default constructor needed by RPC layer/SecretManager.]]>
+      </doc>
+    </constructor>
+    <method name="getContainerID" return="org.apache.hadoop.yarn.api.records.ContainerId"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getApplicationSubmitter" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getNmHostAddress" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getResource" return="org.apache.hadoop.yarn.api.records.Resource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getExpiryTimeStamp" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getMasterKeyId" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getPriority" return="org.apache.hadoop.yarn.api.records.Priority"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getCreationTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getRMIdentifier" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the RMIdentifier of RM in which containers are allocated
+ @return RMIdentifier]]>
+      </doc>
+    </method>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos.ContainerTokenIdentifierProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getLogAggregationContext" return="org.apache.hadoop.yarn.api.records.LogAggregationContext"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="write"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="out" type="java.io.DataOutput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="readFields"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.DataInput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getKind" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getUser" return="org.apache.hadoop.security.UserGroupInformation"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="other" type="java.lang.Object"/>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="KIND" type="org.apache.hadoop.io.Text"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[TokenIdentifier for a container. Encodes {@link ContainerId},
+ {@link Resource} needed by the container and the target NMs host-address.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.ContainerTokenIdentifier -->
+  <!-- start class org.apache.hadoop.yarn.security.ContainerTokenSelector -->
+  <class name="ContainerTokenSelector" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.security.token.TokenSelector"/>
+    <constructor name="ContainerTokenSelector"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="selectToken" return="org.apache.hadoop.security.token.Token"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="service" type="org.apache.hadoop.io.Text"/>
+      <param name="tokens" type="java.util.Collection"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.ContainerTokenSelector -->
+  <!-- start class org.apache.hadoop.yarn.security.NMTokenIdentifier -->
+  <class name="NMTokenIdentifier" extends="org.apache.hadoop.security.token.TokenIdentifier"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NMTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId, org.apache.hadoop.yarn.api.records.NodeId, java.lang.String, int"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="NMTokenIdentifier"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Default constructor needed by RPC/Secret manager]]>
+      </doc>
+    </constructor>
+    <method name="getApplicationAttemptId" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getNodeId" return="org.apache.hadoop.yarn.api.records.NodeId"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getApplicationSubmitter" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getKeyId" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[44/51] [abbrv] hadoop git commit: YARN-4989. TestWorkPreservingRMRestart#testCapacitySchedulerRecovery fails intermittently. Contributed by Ajith S.

Posted by vv...@apache.org.
YARN-4989. TestWorkPreservingRMRestart#testCapacitySchedulerRecovery fails intermittently. Contributed by Ajith S.


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

Branch: refs/heads/YARN-3926
Commit: 28b66ae919e348123f4c05a4787c9ec56c087c25
Parents: 49b4064
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Mon Jun 13 11:09:32 2016 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Mon Jun 13 11:09:32 2016 +0530

----------------------------------------------------------------------
 .../yarn/server/resourcemanager/TestWorkPreservingRMRestart.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/28b66ae9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.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/TestWorkPreservingRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
index 7e5915b..3a60e02 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
@@ -666,7 +666,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     // Wait for RM to settle down on recovering containers;
     waitForNumContainersToRecover(2, rm2, am1_1.getApplicationAttemptId());
     waitForNumContainersToRecover(2, rm2, am1_2.getApplicationAttemptId());
-    waitForNumContainersToRecover(2, rm2, am1_2.getApplicationAttemptId());
+    waitForNumContainersToRecover(2, rm2, am2.getApplicationAttemptId());
 
     // Calculate each queue's resource usage.
     Resource containerResource = Resource.newInstance(1024, 1);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[21/51] [abbrv] hadoop git commit: HADOOP-12666. Support Microsoft Azure Data Lake - as a file system in Hadoop. Contributed by Vishwajeet Dusane.

Posted by vv...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPostOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPostOpParam.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPostOpParam.java
new file mode 100644
index 0000000..7f7e749
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPostOpParam.java
@@ -0,0 +1,97 @@
+/*
+ * 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.web.resources;
+
+import java.net.HttpURLConnection;
+
+/**
+ * Extended Webhdfs PostOpParam to avoid redirect during append operation for
+ * azure data lake storage.
+ */
+
+public class ADLPostOpParam extends HttpOpParam<ADLPostOpParam.Op> {
+  private static final Domain<Op> DOMAIN = new Domain<ADLPostOpParam.Op>(NAME,
+      Op.class);
+
+  /**
+   * Constructor.
+   *
+   * @param str a string representation of the parameter value.
+   */
+  public ADLPostOpParam(final String str) {
+    super(DOMAIN, DOMAIN.parse(str));
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+
+  /**
+   * Post operations.
+   */
+  public static enum Op implements HttpOpParam.Op {
+    APPEND(true, false, HttpURLConnection.HTTP_OK);
+
+    private final boolean redirect;
+    private final boolean doOutput;
+    private final int expectedHttpResponseCode;
+
+    Op(final boolean doOut, final boolean doRedirect,
+        final int expectHttpResponseCode) {
+      this.doOutput = doOut;
+      this.redirect = doRedirect;
+      this.expectedHttpResponseCode = expectHttpResponseCode;
+    }
+
+    @Override
+    public Type getType() {
+      return Type.POST;
+    }
+
+    @Override
+    public boolean getRequireAuth() {
+      return false;
+    }
+
+    @Override
+    public boolean getDoOutput() {
+      return doOutput;
+    }
+
+    @Override
+    public boolean getRedirect() {
+      return redirect;
+    }
+
+    @Override
+    public int getExpectedHttpResponseCode() {
+      return expectedHttpResponseCode;
+    }
+
+    /**
+     * @return a URI query string.
+     */
+    @Override
+    public String toQueryString() {
+      return NAME + "=" + this;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPutOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPutOpParam.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPutOpParam.java
new file mode 100644
index 0000000..d300a1c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPutOpParam.java
@@ -0,0 +1,94 @@
+/*
+ * 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.web.resources;
+
+import java.net.HttpURLConnection;
+
+/**
+ * Extended Webhdfs PutOpParam to avoid redirect during Create operation for
+ * azure data lake storage.
+ */
+public class ADLPutOpParam extends HttpOpParam<ADLPutOpParam.Op> {
+  private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
+
+  /**
+   * Constructor.
+   *
+   * @param str a string representation of the parameter value.
+   */
+  public ADLPutOpParam(final String str) {
+    super(DOMAIN, DOMAIN.parse(str));
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+
+  /**
+   * Put operations.
+   */
+  public static enum Op implements HttpOpParam.Op {
+    CREATE(true, false, HttpURLConnection.HTTP_CREATED);
+
+    private final boolean redirect;
+    private final boolean doOutput;
+    private final int expectedHttpResponseCode;
+    private final boolean requireAuth;
+
+    Op(final boolean doOut, final boolean doRedirect,
+        final int expectHttpResponseCode) {
+      this.doOutput = doOut;
+      this.redirect = doRedirect;
+      this.expectedHttpResponseCode = expectHttpResponseCode;
+      this.requireAuth = false;
+    }
+
+    @Override
+    public HttpOpParam.Type getType() {
+      return HttpOpParam.Type.PUT;
+    }
+
+    @Override
+    public boolean getRequireAuth() {
+      return requireAuth;
+    }
+
+    @Override
+    public boolean getDoOutput() {
+      return doOutput;
+    }
+
+    @Override
+    public boolean getRedirect() {
+      return redirect;
+    }
+
+    @Override
+    public int getExpectedHttpResponseCode() {
+      return expectedHttpResponseCode;
+    }
+
+    @Override
+    public String toQueryString() {
+      return NAME + "=" + this;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLVersionInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLVersionInfo.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLVersionInfo.java
new file mode 100644
index 0000000..0bfe521
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLVersionInfo.java
@@ -0,0 +1,51 @@
+/*
+ * 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.web.resources;
+
+import org.apache.hadoop.hdfs.web.ADLConfKeys;
+
+import java.util.regex.Pattern;
+
+/**
+ * Capture ADL Jar version information. Require for debugging and analysis
+ * purpose in the backend.
+ */
+public class ADLVersionInfo extends StringParam {
+  /**
+   * Parameter name.
+   */
+  public static final String NAME = ADLConfKeys.ADL_WEBSDK_VERSION_KEY;
+
+  private static final StringParam.Domain DOMAIN = new StringParam.Domain(NAME,
+      Pattern.compile(".+"));
+
+  /**
+   * Constructor.
+   * @param featureSetVersion Enabled featured information
+   */
+  public ADLVersionInfo(String featureSetVersion) {
+    super(DOMAIN, featureSetVersion);
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/AppendADLNoRedirectParam.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/AppendADLNoRedirectParam.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/AppendADLNoRedirectParam.java
new file mode 100644
index 0000000..b9ea79e
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/AppendADLNoRedirectParam.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+/**
+ * Overwrite parameter.
+ */
+public class AppendADLNoRedirectParam extends BooleanParam {
+  /**
+   * Parameter name.
+   */
+  public static final String NAME = "append";
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   *
+   * @param value the parameter value.
+   */
+  public AppendADLNoRedirectParam(final Boolean value) {
+    super(DOMAIN, value);
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateADLNoRedirectParam.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateADLNoRedirectParam.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateADLNoRedirectParam.java
new file mode 100644
index 0000000..83f3970
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateADLNoRedirectParam.java
@@ -0,0 +1,44 @@
+/**
+ * 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.web.resources;
+
+/**
+ * Overwrite parameter.
+ */
+public class CreateADLNoRedirectParam extends BooleanParam {
+  /**
+   * Parameter name.
+   */
+  public static final String NAME = "write";
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   *
+   * @param value the parameter value.
+   */
+  public CreateADLNoRedirectParam(final Boolean value) {
+    super(DOMAIN, value);
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/LeaseParam.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/LeaseParam.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/LeaseParam.java
new file mode 100644
index 0000000..6801235
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/LeaseParam.java
@@ -0,0 +1,53 @@
+/*
+ * 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.web.resources;
+
+/**
+ * To support single writer semantics. Notify to ADL backend if the stream
+ * needs to locked in order to protect
+ * concurrent write operation on the same stream.
+ *
+ * Used in append operation.
+ */
+public class LeaseParam extends StringParam {
+
+  public static final String NAME = "leaseId";
+  /**
+   * Default parameter value.
+   */
+  public static final String DEFAULT = NULL;
+
+  private static final StringParam.Domain DOMAIN = new StringParam.Domain(NAME,
+      null);
+
+  /**
+   * Constructor.
+   *
+   * @param str a string representation of the parameter value.
+   */
+  public LeaseParam(final String str) {
+    super(DOMAIN, str == null || str.equals(DEFAULT) ? null : str);
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ReadADLNoRedirectParam.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ReadADLNoRedirectParam.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ReadADLNoRedirectParam.java
new file mode 100644
index 0000000..a600161
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ReadADLNoRedirectParam.java
@@ -0,0 +1,44 @@
+/**
+ * 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.web.resources;
+
+/**
+ * Overwrite parameter.
+ */
+public class ReadADLNoRedirectParam extends BooleanParam {
+  /**
+   * Parameter name.
+   */
+  public static final String NAME = "read";
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   *
+   * @param value the parameter value.
+   */
+  public ReadADLNoRedirectParam(final Boolean value) {
+    super(DOMAIN, value);
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/package-info.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/package-info.java
new file mode 100644
index 0000000..2231cc2
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ *
+ */
+
+/**
+ * A distributed implementation of {@link
+ * org.apache.hadoop.hdfs.web.resources} for reading or extending query
+ * parameter for webhdfs specification. ADL
+ * specific
+ * query parameter also goes in the same package.
+ */
+package org.apache.hadoop.hdfs.web.resources;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md b/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
new file mode 100644
index 0000000..4158c88
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
@@ -0,0 +1,219 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# Hadoop Azure Data Lake Support
+
+* [Introduction](#Introduction)
+* [Features](#Features)
+* [Limitations](#Limitations)
+* [Usage](#Usage)
+    * [Concepts](#Concepts)
+        * [Webhdfs Compliance](#Webhdfs_Specification_Compliance)
+        * [OAuth2 Support](#OAuth2_Support)
+        * [Read Ahead Buffer Management](Read_Ahead_Buffer_Management)
+    * [Configuring Credentials & FileSystem](#Configuring_Credentials)
+    * [Accessing adl URLs](#Accessing_adl_URLs)
+* [Testing the hadoop-azure Module](#Testing_the_hadoop-azure_Module)
+
+## <a name="Introduction" />Introduction
+
+The hadoop-azure-datalake module provides support for integration with
+[Azure Data Lake Store]( https://azure.microsoft.com/en-in/documentation/services/data-lake-store/).
+The jar file is named azure-datalake-store.jar.
+
+## <a name="Features" />Features
+
+* Read and write data stored in an Azure Data Lake Storage account.
+* Partial support for [Webhdfs Specification 2.7.0](https://hadoop.apache.org/docs/r2.7.0/hadoop-project-dist/hadoop-hdfs/WebHDFS.html)
+* Reference file system paths using URLs using the `adl` scheme for Secure Webhdfs i.e. SSL
+  encrypted access.
+* Can act as a source of data in a MapReduce job, or a sink.
+* Tested on both Linux and Windows.
+* Tested for scale.
+
+## <a name="Limitations" />Limitations
+Partial or no support for the following operations in [Webhdfs Specification 2.7.0](https://hadoop.apache.org/docs/r2.7.0/hadoop-project-dist/hadoop-hdfs/WebHDFS.html):
+
+* Operation on Symbolic Link
+* Proxy Users
+* File Truncate
+* File Checksum
+* File replication factor
+* Home Directory Partial supported based on OAuth2 token information and not the active user on Hadoop cluster.
+* Extended Attributes(XAttrs) Operations
+* Snapshot Operations
+* Delegation Token Operations
+* User and group information returned as ListStatus and GetFileStatus is in form of GUID associated in Azure Active Directory.
+
+## <a name="Usage" />Usage
+
+### <a name="Concepts" />Concepts
+Azure Data Lake Storage access path syntax is
+
+    adl://<Account Name>.azuredatalakestore.net/
+
+Get started with azure data lake account with [https://azure.microsoft.com/en-in/documentation/articles/data-lake-store-get-started-portal/](https://azure.microsoft.com/en-in/documentation/articles/data-lake-store-get-started-portal/)
+
+#### <a name="Webhdfs_Specification_Compliance" />Webhdfs Compliance
+Azure Data Lake Storage exposes a public REST endpoint as per [Webhdfs Specification 2.7.0](https://hadoop.apache.org/docs/r2.7.0/hadoop-project-dist/hadoop-hdfs/WebHDFS.html) to access storage file system.
+
+Syntax to access Azure data lake storage account over [Webhdfs Specification 2.7.0](https://hadoop.apache.org/docs/r2.7.0/hadoop-project-dist/hadoop-hdfs/WebHDFS.html) is
+
+    https://<Account Name>.azuredatalakestore.net/webhdfs/v1/<File System Path>?<Query paramaters>
+
+
+#### <a name="#OAuth2_Support" />OAuth2 Support
+Usage of Azure Data Lake Storage requires OAuth2 bearer token to be present as part of the HTTPS header as per OAuth2 specification. Valid OAuth2 bearer token should be obtained from Azure Active Directory for valid users who have  access to Azure Data Lake Storage Account.
+
+Azure Active Directory (Azure AD) is Microsoft\ufffds multi-tenant cloud based directory and identity management service. See [https://azure.microsoft.com/en-in/documentation/articles/active-directory-whatis/](https://azure.microsoft.com/en-in/documentation/articles/active-directory-whatis/)
+
+Following sections describes on OAuth2 configuration in core-site.xml.
+
+#### <a name="#Read_Ahead_Buffer_Management" />Read Ahead Buffer Management
+Azure Data Lake Storage offers high throughput. To maximize throughput, applications can  use this feature to buffer data concurrently, in memory during read operation. This data is cached in memory per process per stream.
+
+
+To Enable/Disable read ahead feature.
+
+    <property>
+        <name>adl.feature.override.readahead</name>
+        <value>true</value>
+        <description>
+            Enables read aheads in the ADL client, the feature is used to improve read throughput.
+            This works in conjunction with the value set in adl.feature.override.readahead.max.buffersize.
+            When set to false the read ahead feature is turned off.
+            Default : True if not configured.
+        </description>
+    </property>
+
+To configure read ahead buffer size.
+
+    <property>
+        <name>adl.feature.override.readahead.max.buffersize</name>
+        <value>8388608</value>
+        <description>
+            Define maximum buffer size to cache read ahead data, this is allocated per process to
+            cache read ahead data. Applicable only when adl.feature.override.readahead is set to true.
+            Default : 8388608 Byte i.e. 8MB if not configured.
+        </description>
+    </property>
+
+To configure number of concurrent connection to Azure Data Lake Storage Account.
+
+    <property>
+        <name>adl.feature.override.readahead.max.concurrent.connection</name>
+        <value>2</value>
+        <description>
+            Define maximum concurrent connection can be established to
+            read ahead. If the data size is<4MB then only 1 read n/w connection
+            is set. If the data size is >4MB but<8MB then 2 read n/w
+            connection
+            is set. Data >8MB then value set under the property would
+             take
+            effect. Applicable only when adl.feature.override.readahead is set
+            to true and buffer size is >8MB.
+            It is recommended to reset this property if the adl.feature.override.readahead.max.buffersize
+            is < 8MB to gain performance. Application has to consider
+             throttling
+            limit for the account as well before configuring large buffer size.
+        </description>
+    </property>
+
+## <a name="Configuring_Credentials" />Configuring Credentials & FileSystem
+
+Update core-site.xml for OAuth2 configuration
+
+         <property>
+            <name>dfs.webhdfs.oauth2.refresh.token.expires.ms.since.epoch</name>
+            <value>0</value>
+         </property>
+
+         <property>
+            <name>dfs.webhdfs.oauth2.credential</name>
+            <value>bearer.and.refresh.token</value>
+         </property>
+
+        <property>
+            <name>dfs.webhdfs.oauth2.access.token</name>
+            <value>NOT_SET</value>
+        </property>
+
+        <property>
+            <name>dfs.webhdfs.oauth2.refresh.url</name>
+            <value>https://login.windows.net/common/oauth2/token/</value>
+        </property>
+
+        <property>
+            <name>dfs.webhdfs.oauth2.access.token.provider</name>
+            <value>org.apache.hadoop.fs.adl.oauth2.CachedRefreshTokenBasedAccessTokenProvider</value>
+        </property>
+
+Application require to set Client id and OAuth2 refresh token from Azure Active Directory associated with client id. See [https://github.com/AzureAD/azure-activedirectory-library-for-java](https://github.com/AzureAD/azure-activedirectory-library-for-java).
+
+**Do not share client id and refresh token, it must be kept secret.**
+
+        <property>
+            <name>dfs.webhdfs.oauth2.client.id</name>
+            <value></value>
+        </property>
+
+        <property>
+            <name>dfs.webhdfs.oauth2.refresh.token</name>
+            <value></value>
+        </property>
+
+For ADL FileSystem to take effect. Update core-site.xml with
+
+        <property>
+            <name>fs.adl.impl</name>
+            <value>org.apache.hadoop.fs.adl.AdlFileSystem</value>
+        </property>
+
+        <property>
+            <name>fs.AbstractFileSystem.adl.impl</name>
+            <value>org.apache.hadoop.fs.adl.Adl</value>
+        </property>
+
+
+### <a name="Accessing_adl_URLs" />Accessing adl URLs
+
+After credentials are configured in core-site.xml, any Hadoop component may
+reference files in that Azure Data Lake Storage account by using URLs of the following
+format:
+
+    adl://<Account Name>.azuredatalakestore.net/<path>
+
+The schemes `adl` identify a URL on a file system backed by Azure
+Data Lake Storage.  `adl` utilizes encrypted HTTPS access for all interaction with
+the Azure Data Lake Storage API.
+
+For example, the following
+[FileSystem Shell](../hadoop-project-dist/hadoop-common/FileSystemShell.html)
+commands demonstrate access to a storage account named `youraccount`.
+
+    > hadoop fs -mkdir adl://yourcontainer.azuredatalakestore.net/testDir
+
+    > hadoop fs -put testFile adl://yourcontainer.azuredatalakestore.net/testDir/testFile
+
+    > hadoop fs -cat adl://yourcontainer.azuredatalakestore.net/testDir/testFile
+    test file content
+## <a name="Testing_the_hadoop-azure_Module" />Testing the azure-datalake-store Module
+The hadoop-azure module includes a full suite of unit tests. Most of the tests will run without additional configuration by running mvn test. This includes tests against mocked storage, which is an in-memory emulation of Azure Data Lake Storage.
+
+A selection of tests can run against the Azure Data Lake Storage. To run tests against Adl storage. Please configure contract-test-options.xml with Adl account information mentioned in the above sections. Also turn on contract test execution flag to trigger tests against Azure Data Lake Storage.
+
+    <property>
+      <name>dfs.adl.test.contract.enable</name>
+      <value>true</value>
+    </property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/oauth2/TestCachedRefreshTokenBasedAccessTokenProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/oauth2/TestCachedRefreshTokenBasedAccessTokenProvider.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/oauth2/TestCachedRefreshTokenBasedAccessTokenProvider.java
new file mode 100644
index 0000000..e57d3a9
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/oauth2/TestCachedRefreshTokenBasedAccessTokenProvider.java
@@ -0,0 +1,147 @@
+/*
+ * 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.adl.oauth2;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.web.oauth2.AccessTokenProvider;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY;
+import static org.apache.hadoop.hdfs.web.oauth2.ConfRefreshTokenBasedAccessTokenProvider.OAUTH_REFRESH_TOKEN_KEY;
+
+/**
+ * Verify cache behavior of ConfRefreshTokenBasedAccessTokenProvider instances.
+ */
+public class TestCachedRefreshTokenBasedAccessTokenProvider {
+
+  private Configuration conf;
+
+  @Rule public TestName name = new TestName();
+  String clientId(int id) {
+    return name.getMethodName() + "_clientID" + id;
+  }
+
+  @Before
+  public void initConfig() {
+    conf = new Configuration(false);
+    conf.set(OAUTH_CLIENT_ID_KEY, clientId(0));
+    conf.set(OAUTH_REFRESH_TOKEN_KEY, "01234567890abcdef");
+    conf.set(OAUTH_REFRESH_URL_KEY, "http://dingo.invalid:80");
+  }
+
+  @Test
+  public void testCacheInstance() throws Exception {
+    final AccessTokenProvider inst0 = mock(AccessTokenProvider.class);
+    when(inst0.getConf()).thenReturn(conf);
+
+    // verify config
+    CachedRefreshTokenBasedAccessTokenProvider t1 = new MockProvider(inst0);
+    t1.setConf(conf);
+    verify(inst0).setConf(any(Configuration.class)); // cloned, not exact match
+
+    // verify cache hit
+    CachedRefreshTokenBasedAccessTokenProvider t2 =
+        new CachedRefreshTokenBasedAccessTokenProvider() {
+          @Override
+          AccessTokenProvider newInstance() {
+            fail("Failed to return cached instance");
+            return null;
+          }
+        };
+    t2.setConf(conf);
+
+    // verify force refresh
+    conf.setBoolean(
+        CachedRefreshTokenBasedAccessTokenProvider.FORCE_REFRESH, true);
+    final AccessTokenProvider inst1 = mock(AccessTokenProvider.class);
+    when(inst1.getConf()).thenReturn(conf);
+    CachedRefreshTokenBasedAccessTokenProvider t3 = new MockProvider(inst1);
+    t3.setConf(conf);
+    verify(inst1).setConf(any(Configuration.class));
+
+    // verify cache miss
+    conf.set(OAUTH_REFRESH_URL_KEY, "http://yak.invalid:80");
+    final AccessTokenProvider inst2 = mock(AccessTokenProvider.class);
+    when(inst2.getConf()).thenReturn(conf);
+    CachedRefreshTokenBasedAccessTokenProvider t4 = new MockProvider(inst2);
+    t4.setConf(conf);
+    verify(inst2).setConf(any(Configuration.class));
+  }
+
+  @Test
+  public void testCacheLimit() throws Exception {
+    final int iter = CachedRefreshTokenBasedAccessTokenProvider.MAX_PROVIDERS;
+    for (int i = 0; i < iter; ++i) {
+      conf.set(OAUTH_CLIENT_ID_KEY, clientId(i));
+      AccessTokenProvider inst = mock(AccessTokenProvider.class);
+      when(inst.getConf()).thenReturn(conf);
+      CachedRefreshTokenBasedAccessTokenProvider t = new MockProvider(inst);
+      t.setConf(conf);
+      verify(inst).setConf(any(Configuration.class));
+    }
+    // verify cache hit
+    for (int i = 0; i < iter; ++i) {
+      conf.set(OAUTH_CLIENT_ID_KEY, clientId(i));
+      CachedRefreshTokenBasedAccessTokenProvider t =
+          new CachedRefreshTokenBasedAccessTokenProvider() {
+            @Override
+            AccessTokenProvider newInstance() {
+              fail("Failed to return cached instance");
+              return null;
+            }
+          };
+      t.setConf(conf);
+    }
+
+    // verify miss, evict 0
+    conf.set(OAUTH_CLIENT_ID_KEY, clientId(iter));
+    final AccessTokenProvider inst = mock(AccessTokenProvider.class);
+    when(inst.getConf()).thenReturn(conf);
+    CachedRefreshTokenBasedAccessTokenProvider t = new MockProvider(inst);
+    t.setConf(conf);
+    verify(inst).setConf(any(Configuration.class));
+
+    // verify miss
+    conf.set(OAUTH_CLIENT_ID_KEY, clientId(0));
+    final AccessTokenProvider inst0 = mock(AccessTokenProvider.class);
+    when(inst0.getConf()).thenReturn(conf);
+    CachedRefreshTokenBasedAccessTokenProvider t0 = new MockProvider(inst0);
+    t0.setConf(conf);
+    verify(inst0).setConf(any(Configuration.class));
+  }
+
+  static class MockProvider extends CachedRefreshTokenBasedAccessTokenProvider {
+    private final AccessTokenProvider inst;
+    MockProvider(AccessTokenProvider inst) {
+      this.inst = inst;
+    }
+    @Override
+    AccessTokenProvider newInstance() {
+      return inst;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-tools-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-tools-dist/pom.xml b/hadoop-tools/hadoop-tools-dist/pom.xml
index e1fbef1..9485741 100644
--- a/hadoop-tools/hadoop-tools-dist/pom.xml
+++ b/hadoop-tools/hadoop-tools-dist/pom.xml
@@ -105,6 +105,12 @@
       <artifactId>hadoop-sls</artifactId>
       <scope>compile</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-azure-datalake</artifactId>
+      <scope>compile</scope>
+      <version>${project.version}</version>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/pom.xml b/hadoop-tools/pom.xml
index bd5f784..a2c441b 100644
--- a/hadoop-tools/pom.xml
+++ b/hadoop-tools/pom.xml
@@ -46,6 +46,7 @@
     <module>hadoop-azure</module>
     <module>hadoop-aws</module>
     <module>hadoop-kafka</module>
+    <module>hadoop-azure-datalake</module>
   </modules>
 
   <build>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[45/51] [abbrv] hadoop git commit: YARN-4081. Add support for multiple resource types in the Resource class. (Varun Vasudev via wangda)

Posted by vv...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e518ef1/hadoop-build-tools/src/main/resources/META-INF/NOTICE.txt
----------------------------------------------------------------------
diff --git a/hadoop-build-tools/src/main/resources/META-INF/NOTICE.txt b/hadoop-build-tools/src/main/resources/META-INF/NOTICE.txt
new file mode 100644
index 0000000..63fbc9d
--- /dev/null
+++ b/hadoop-build-tools/src/main/resources/META-INF/NOTICE.txt
@@ -0,0 +1,283 @@
+This product includes software developed by The Apache Software
+Foundation (http://www.apache.org/).
+
+The binary distribution of this product bundles binaries of
+org.iq80.leveldb:leveldb-api (https://github.com/dain/leveldb), which has the
+following notices:
+* Copyright 2011 Dain Sundstrom <da...@iq80.com>
+* Copyright 2011 FuseSource Corp. http://fusesource.com
+
+The binary distribution of this product bundles binaries of
+org.fusesource.hawtjni:hawtjni-runtime (https://github.com/fusesource/hawtjni),
+which has the following notices:
+* This product includes software developed by FuseSource Corp.
+  http://fusesource.com
+* This product includes software developed at
+  Progress Software Corporation and/or its  subsidiaries or affiliates.
+* This product includes software developed by IBM Corporation and others.
+
+The binary distribution of this product bundles binaries of
+AWS Java SDK 1.10.6,
+which has the following notices:
+ * This software includes third party software subject to the following
+ copyrights: - XML parsing and utility functions from JetS3t - Copyright
+ 2006-2009 James Murty. - JSON parsing and utility functions from JSON.org -
+ Copyright 2002 JSON.org. - PKCS#1 PEM encoded private key parsing and utility
+ functions from oauth.googlecode.com - Copyright 1998-2010 AOL Inc.
+
+The binary distribution of this product bundles binaries of
+Gson 2.2.4,
+which has the following notices:
+
+                            The Netty Project
+                            =================
+
+Please visit the Netty web site for more information:
+
+  * http://netty.io/
+
+Copyright 2014 The Netty Project
+
+The Netty Project 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.
+
+Also, please refer to each LICENSE.<component>.txt file, which is located in
+the 'license' directory of the distribution file, for the license terms of the
+components that this product depends on.
+
+-------------------------------------------------------------------------------
+This product contains the extensions to Java Collections Framework which has
+been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
+
+  * LICENSE:
+    * license/LICENSE.jsr166y.txt (Public Domain)
+  * HOMEPAGE:
+    * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
+    * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
+
+This product contains a modified version of Robert Harder's Public Domain
+Base64 Encoder and Decoder, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.base64.txt (Public Domain)
+  * HOMEPAGE:
+    * http://iharder.sourceforge.net/current/java/base64/
+
+This product contains a modified portion of 'Webbit', an event based
+WebSocket and HTTP server, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.webbit.txt (BSD License)
+  * HOMEPAGE:
+    * https://github.com/joewalnes/webbit
+
+This product contains a modified portion of 'SLF4J', a simple logging
+facade for Java, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.slf4j.txt (MIT License)
+  * HOMEPAGE:
+    * http://www.slf4j.org/
+
+This product contains a modified portion of 'ArrayDeque', written by Josh
+Bloch of Google, Inc:
+
+  * LICENSE:
+    * license/LICENSE.deque.txt (Public Domain)
+
+This product contains a modified portion of 'Apache Harmony', an open source
+Java SE, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.harmony.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://archive.apache.org/dist/harmony/
+
+This product contains a modified version of Roland Kuhn's ASL2
+AbstractNodeQueue, which is based on Dmitriy Vyukov's non-intrusive MPSC queue.
+It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.abstractnodequeue.txt (Public Domain)
+  * HOMEPAGE:
+    * https://github.com/akka/akka/blob/wip-2.2.3-for-scala-2.11/akka-actor/src/main/java/akka/dispatch/AbstractNodeQueue.java
+
+This product contains a modified portion of 'jbzip2', a Java bzip2 compression
+and decompression library written by Matthew J. Francis. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jbzip2.txt (MIT License)
+  * HOMEPAGE:
+    * https://code.google.com/p/jbzip2/
+
+This product contains a modified portion of 'libdivsufsort', a C API library to construct
+the suffix array and the Burrows-Wheeler transformed string for any input string of
+a constant-size alphabet written by Yuta Mori. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.libdivsufsort.txt (MIT License)
+  * HOMEPAGE:
+    * https://code.google.com/p/libdivsufsort/
+
+This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
+ which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jctools.txt (ASL2 License)
+  * HOMEPAGE:
+    * https://github.com/JCTools/JCTools
+
+This product optionally depends on 'JZlib', a re-implementation of zlib in
+pure Java, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jzlib.txt (BSD style License)
+  * HOMEPAGE:
+    * http://www.jcraft.com/jzlib/
+
+This product optionally depends on 'Compress-LZF', a Java library for encoding and
+decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.compress-lzf.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/ning/compress
+
+This product optionally depends on 'lz4', a LZ4 Java compression
+and decompression library written by Adrien Grand. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.lz4.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/jpountz/lz4-java
+
+This product optionally depends on 'lzma-java', a LZMA Java compression
+and decompression library, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.lzma-java.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/jponge/lzma-java
+
+This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
+and decompression library written by William Kinney. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jfastlz.txt (MIT License)
+  * HOMEPAGE:
+    * https://code.google.com/p/jfastlz/
+
+This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
+interchange format, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.protobuf.txt (New BSD License)
+  * HOMEPAGE:
+    * http://code.google.com/p/protobuf/
+
+This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
+a temporary self-signed X.509 certificate when the JVM does not provide the
+equivalent functionality.  It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.bouncycastle.txt (MIT License)
+  * HOMEPAGE:
+    * http://www.bouncycastle.org/
+
+This product optionally depends on 'Snappy', a compression library produced
+by Google Inc, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.snappy.txt (New BSD License)
+  * HOMEPAGE:
+    * http://code.google.com/p/snappy/
+
+This product optionally depends on 'JBoss Marshalling', an alternative Java
+serialization API, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1)
+  * HOMEPAGE:
+    * http://www.jboss.org/jbossmarshalling
+
+This product optionally depends on 'Caliper', Google's micro-
+benchmarking framework, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.caliper.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://code.google.com/p/caliper/
+
+This product optionally depends on 'Apache Commons Logging', a logging
+framework, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.commons-logging.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://commons.apache.org/logging/
+
+This product optionally depends on 'Apache Log4J', a logging framework, which
+can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.log4j.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://logging.apache.org/log4j/
+
+This product optionally depends on 'Aalto XML', an ultra-high performance
+non-blocking XML processor, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.aalto-xml.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://wiki.fasterxml.com/AaltoHome
+
+This product contains a modified version of 'HPACK', a Java implementation of
+the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.hpack.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/twitter/hpack
+
+This product contains a modified portion of 'Apache Commons Lang', a Java library
+provides utilities for the java.lang API, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.commons-lang.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://commons.apache.org/proper/commons-lang/
+
+The binary distribution of this product bundles binaries of
+Commons Codec 1.4,
+which has the following notices:
+ * src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.javacontains test data from http://aspell.net/test/orig/batch0.tab.Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org)
+  ===============================================================================
+  The content of package org.apache.commons.codec.language.bm has been translated
+  from the original php source code available at http://stevemorse.org/phoneticinfo.htm
+  with permission from the original authors.
+  Original source copyright:Copyright (c) 2008 Alexander Beider & Stephen P. Morse.
+
+The binary distribution of this product bundles binaries of
+Commons Lang 2.6,
+which has the following notices:
+ * This product includes software from the Spring Framework,under the Apache License 2.0 (see: StringUtils.containsWhitespace())
+
+The binary distribution of this product bundles binaries of
+Apache Log4j 1.2.17,
+which has the following notices:
+ * ResolverUtil.java
+    Copyright 2005-2006 Tim Fennell
+  Dumbster SMTP test server
+    Copyright 2004 Jason Paul Kitchen
+  TypeUtil.java
+    Copyright 2002-2012 Ramnivas Laddad, Juergen Hoeller, Chris Beams
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e518ef1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ResourceTypes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ResourceTypes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ResourceTypes.java
new file mode 100644
index 0000000..dbd9c37
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/ResourceTypes.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.protocolrecords;
+
+/**
+ * Enum which represents the resource type. Currently, the only type allowed is
+ * COUNTABLE.
+ */
+public enum ResourceTypes {
+  COUNTABLE
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e518ef1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
index e2d8472..7fca147 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
@@ -24,8 +24,12 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
+import org.apache.hadoop.yarn.exceptions.ResourceNotFoundException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.Records;
 
+import java.util.Map;
+
 /**
  * <p><code>Resource</code> models a set of computer resources in the 
  * cluster.</p>
@@ -39,10 +43,10 @@ import org.apache.hadoop.yarn.util.Records;
  * the average number of threads it expects to have runnable at a time.</p>
  * 
  * <p>Virtual cores take integer values and thus currently CPU-scheduling is
- * very coarse.  A complementary axis for CPU requests that represents processing
- * power will likely be added in the future to enable finer-grained resource
- * configuration.</p>
- * 
+ * very coarse.  A complementary axis for CPU requests that represents
+ * processing power will likely be added in the future to enable finer-grained
+ * resource configuration.</p>
+ *
  * <p>Typically, applications request <code>Resource</code> of suitable
  * capability to run their component tasks.</p>
  * 
@@ -62,19 +66,40 @@ public abstract class Resource implements Comparable<Resource> {
     return resource;
   }
 
+  @Public
+  @Stable
+  public static Resource newInstance(
+      Map<String, ResourceInformation> resources) {
+    Resource resource = Records.newRecord(Resource.class);
+    resource.setResources(resources);
+    return resource;
+  }
+
   /**
    * This method is DEPRECATED:
    * Use {@link Resource#getMemorySize()} instead
    *
-   * Get <em>memory</em> of the resource.
-   * @return <em>memory</em> of the resource
+   * Get <em>memory</em> of the resource. Note - while memory has
+   * never had a unit specified, all YARN configurations have specified memory
+   * in MB. The assumption has been that the daemons and applications are always
+   * using the same units. With the introduction of the ResourceInformation
+   * class we have support for units - so this function will continue to return
+   * memory but in the units of MB
+   *
+   * @return <em>memory</em>(in MB) of the resource
    */
   @Public
   @Deprecated
   public abstract int getMemory();
 
   /**
-   * Get <em>memory</em> of the resource.
+   * Get <em>memory</em> of the resource. Note - while memory has
+   * never had a unit specified, all YARN configurations have specified memory
+   * in MB. The assumption has been that the daemons and applications are always
+   * using the same units. With the introduction of the ResourceInformation
+   * class we have support for units - so this function will continue to return
+   * memory but in the units of MB
+   *
    * @return <em>memory</em> of the resource
    */
   @Private
@@ -82,8 +107,14 @@ public abstract class Resource implements Comparable<Resource> {
   public abstract long getMemorySize();
   
   /**
-   * Set <em>memory</em> of the resource.
-   * @param memory <em>memory</em> of the resource
+   * Set <em>memory</em> of the resource. Note - while memory has
+   * never had a unit specified, all YARN configurations have specified memory
+   * in MB. The assumption has been that the daemons and applications are always
+   * using the same units. With the introduction of the ResourceInformation
+   * class we have support for units - so this function will continue to set
+   * memory but the assumption is that the value passed is in units of MB.
+   *
+   * @param memory <em>memory</em>(in MB) of the resource
    */
   @Public
   @Stable
@@ -94,10 +125,11 @@ public abstract class Resource implements Comparable<Resource> {
    * Get <em>number of virtual cpu cores</em> of the resource.
    * 
    * Virtual cores are a unit for expressing CPU parallelism. A node's capacity
-   * should be configured with virtual cores equal to its number of physical cores.
-   * A container should be requested with the number of cores it can saturate, i.e.
-   * the average number of threads it expects to have runnable at a time.
-   *   
+   * should be configured with virtual cores equal to its number of physical
+   * cores. A container should be requested with the number of cores it can
+   * saturate, i.e. the average number of threads it expects to have runnable
+   * at a time.
+   *
    * @return <em>num of virtual cpu cores</em> of the resource
    */
   @Public
@@ -112,16 +144,84 @@ public abstract class Resource implements Comparable<Resource> {
    * Set <em>number of virtual cpu cores</em> of the resource.
    * 
    * Virtual cores are a unit for expressing CPU parallelism. A node's capacity
-   * should be configured with virtual cores equal to its number of physical cores.
-   * A container should be requested with the number of cores it can saturate, i.e.
-   * the average number of threads it expects to have runnable at a time.
-   *    
+   * should be configured with virtual cores equal to its number of physical
+   * cores. A container should be requested with the number of cores it can
+   * saturate, i.e. the average number of threads it expects to have runnable
+   * at a time.
+   *
    * @param vCores <em>number of virtual cpu cores</em> of the resource
    */
   @Public
   @Evolving
   public abstract void setVirtualCores(long vCores);
 
+  /**
+   * Get ResourceInformation for all resources.
+   *
+   * @return Map of resource name to ResourceInformation
+   */
+  @Public
+  @Evolving
+  public abstract Map<String, ResourceInformation> getResources();
+
+  /**
+   * Get ResourceInformation for a specified resource.
+   *
+   * @param resource name of the resource
+   * @return the ResourceInformation object for the resource
+   * @throws YarnException if the resource can't be found
+   */
+  @Public
+  @Evolving
+  public abstract ResourceInformation getResourceInformation(String resource)
+      throws YarnException;
+
+  /**
+   * Get the value for a specified resource. No information about the units is
+   * returned.
+   *
+   * @param resource name of the resource
+   * @return the value for the resource
+   * @throws YarnException if the resource can't be found
+   */
+  @Public
+  @Evolving
+  public abstract Long getResourceValue(String resource) throws YarnException;
+
+  /**
+   * Set the resources to the map specified.
+   *
+   * @param resources Desired resources
+   */
+  @Public
+  @Evolving
+  public abstract void setResources(Map<String, ResourceInformation> resources);
+
+  /**
+   * Set the ResourceInformation object for a particular resource.
+   *
+   * @param resource the resource for which the ResourceInformation is provided
+   * @param resourceInformation ResourceInformation object
+   * @throws ResourceNotFoundException if the resource is not found
+   */
+  @Public
+  @Evolving
+  public abstract void setResourceInformation(String resource,
+      ResourceInformation resourceInformation) throws ResourceNotFoundException;
+
+  /**
+   * Set the value of a resource in the ResourceInformation object. The unit of
+   * the value is assumed to be the one in the ResourceInformation object.
+   *
+   * @param resource the resource for which the value is provided.
+   * @param value    the value to set
+   * @throws ResourceNotFoundException if the resource is not found
+   */
+  @Public
+  @Evolving
+  public abstract void setResourceValue(String resource, Long value)
+      throws ResourceNotFoundException;
+
   @Override
   public int hashCode() {
     final int prime = 263167;
@@ -129,27 +229,84 @@ public abstract class Resource implements Comparable<Resource> {
     int result = (int) (939769357
         + getMemorySize()); // prime * result = 939769357 initially
     result = prime * result + getVirtualCores();
+    for (Map.Entry<String, ResourceInformation> entry : getResources()
+        .entrySet()) {
+      if (entry.getKey().equals(ResourceInformation.MEMORY.getName()) || entry
+          .getKey().equals(ResourceInformation.VCORES.getName())) {
+        continue;
+      }
+      result = prime * result + entry.getValue().hashCode();
+    }
     return result;
   }
 
   @Override
   public boolean equals(Object obj) {
-    if (this == obj)
+    if (this == obj) {
       return true;
-    if (obj == null)
+    }
+    if (obj == null) {
       return false;
-    if (!(obj instanceof Resource))
+    }
+    if (!(obj instanceof Resource)) {
       return false;
+    }
     Resource other = (Resource) obj;
-    if (getMemorySize() != other.getMemorySize() ||
-        getVirtualCores() != other.getVirtualCores()) {
+    if (getMemorySize() != other.getMemorySize() || getVirtualCores() != other
+        .getVirtualCores()) {
       return false;
     }
-    return true;
+    return this.getResources().equals(other.getResources());
   }
 
   @Override
   public String toString() {
-    return "<memory:" + getMemorySize() + ", vCores:" + getVirtualCores() + ">";
+    StringBuilder sb = new StringBuilder();
+    sb.append("<memory:").append(getMemorySize()).append(", vCores:")
+        .append(getVirtualCores());
+    for (Map.Entry<String, ResourceInformation> entry : getResources()
+        .entrySet()) {
+      if (entry.getKey().equals(ResourceInformation.MEMORY.getName())
+          && entry.getValue().getUnits()
+          .equals(ResourceInformation.MEMORY_MB.getUnits())) {
+        continue;
+      }
+      if (entry.getKey().equals(ResourceInformation.VCORES.getName())
+          && entry.getValue().getUnits().equals("")) {
+        continue;
+      }
+      sb.append(", ").append(entry.getKey()).append(": ")
+          .append(entry.getValue().getValue())
+          .append(entry.getValue().getUnits());
+    }
+    sb.append(">");
+    return sb.toString();
+  }
+
+  @Override
+  public int compareTo(Resource other) {
+    Map<String, ResourceInformation> thisResources, otherResources;
+    thisResources = this.getResources();
+    otherResources = other.getResources();
+    long diff = thisResources.size() - otherResources.size();
+    if (diff == 0) {
+      if (thisResources.keySet().equals(otherResources.keySet())) {
+        diff = this.getMemorySize() - other.getMemorySize();
+        if (diff == 0) {
+          diff = this.getVirtualCores() - other.getVirtualCores();
+        }
+        if (diff == 0) {
+          for (Map.Entry<String, ResourceInformation> entry : thisResources
+              .entrySet()) {
+            diff =
+                entry.getValue().compareTo(otherResources.get(entry.getKey()));
+            if (diff != 0) {
+              break;
+            }
+          }
+        }
+      }
+    }
+    return Long.compare(diff, 0);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e518ef1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
new file mode 100644
index 0000000..4e780c1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
@@ -0,0 +1,218 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.records;
+
+import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
+import org.apache.hadoop.yarn.util.UnitsConversionUtil;
+
+/**
+ * Class to encapsulate information about a Resource - the name of the resource,
+ * the units(milli, micro, etc), the type(countable), and the value.
+ */
+public class ResourceInformation implements Comparable<ResourceInformation> {
+
+  private String name;
+  private String units;
+  private ResourceTypes resourceType;
+  private Long value;
+
+  private static final String MEMORY_URI = "yarn.io/memory";
+  private static final String VCORES_URI = "yarn.io/vcores";
+
+  public static final ResourceInformation MEMORY =
+      ResourceInformation.newInstance(MEMORY_URI);
+  public static final ResourceInformation MEMORY_MB =
+      ResourceInformation.newInstance(MEMORY_URI, "M");
+  public static final ResourceInformation VCORES =
+      ResourceInformation.newInstance(VCORES_URI);
+
+  /**
+   * Get the name for the resource.
+   *
+   * @return resource name
+   */
+  public String getName() {
+    return name;
+  }
+
+  /**
+   * Set the name for the resource.
+   *
+   * @param rName name for the resource
+   */
+  public void setName(String rName) {
+    this.name = rName;
+  }
+
+  /**
+   * Get units for the resource.
+   *
+   * @return units for the resource
+   */
+  public String getUnits() {
+    return units;
+  }
+
+  /**
+   * Set the units for the resource.
+   *
+   * @param rUnits units for the resource
+   */
+  public void setUnits(String rUnits) {
+    if (!UnitsConversionUtil.KNOWN_UNITS.contains(rUnits)) {
+      throw new IllegalArgumentException(
+          "Unknown unit '" + units + "'. Known units are "
+              + UnitsConversionUtil.KNOWN_UNITS);
+    }
+    this.units = rUnits;
+  }
+
+  /**
+   * Get the resource type.
+   *
+   * @return the resource type
+   */
+  public ResourceTypes getResourceType() {
+    return resourceType;
+  }
+
+  /**
+   * Set the resource type.
+   *
+   * @param type the resource type
+   */
+  public void setResourceType(ResourceTypes type) {
+    this.resourceType = type;
+  }
+
+  /**
+   * Get the value for the resource.
+   *
+   * @return the resource value
+   */
+  public Long getValue() {
+    return value;
+  }
+
+  /**
+   * Set the value for the resource.
+   *
+   * @param rValue the resource value
+   */
+  public void setValue(Long rValue) {
+    this.value = rValue;
+  }
+
+  /**
+   * Create a new instance of ResourceInformation from another object.
+   *
+   * @param other the object from which the new object should be created
+   * @return the new ResourceInformation object
+   */
+  public static ResourceInformation newInstance(ResourceInformation other) {
+    ResourceInformation ret = new ResourceInformation();
+    ret.setName(other.getName());
+    ret.setResourceType(other.getResourceType());
+    ret.setUnits(other.getUnits());
+    ret.setValue(other.getValue());
+    return ret;
+  }
+
+  public static ResourceInformation newInstance(String name, String units,
+      Long value, ResourceTypes type) {
+    ResourceInformation ret = new ResourceInformation();
+    ret.setName(name);
+    ret.setResourceType(type);
+    ret.setUnits(units);
+    ret.setValue(value);
+    return ret;
+  }
+
+  public static ResourceInformation newInstance(String name, String units,
+      Long value) {
+    return ResourceInformation
+        .newInstance(name, units, value, ResourceTypes.COUNTABLE);
+  }
+
+  public static ResourceInformation newInstance(String name, String units) {
+    return ResourceInformation
+        .newInstance(name, units, 0L, ResourceTypes.COUNTABLE);
+  }
+
+  public static ResourceInformation newInstance(String name, Long value) {
+    return ResourceInformation
+        .newInstance(name, "", value, ResourceTypes.COUNTABLE);
+  }
+
+  public static ResourceInformation newInstance(String name) {
+    return ResourceInformation.newInstance(name, "");
+  }
+
+  @Override
+  public String toString() {
+    return "name: " + this.name + ", units: " + this.units + ", type: "
+        + resourceType + ", value: " + value;
+  }
+
+  public String getShorthandRepresentation() {
+    return "" + this.value + this.units;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (!(obj instanceof ResourceInformation)) {
+      return false;
+    }
+    ResourceInformation r = (ResourceInformation) obj;
+    int cmp =
+        UnitsConversionUtil.compare(this.units, this.value, r.units, r.value);
+    return this.name.equals(r.getName()) && this.resourceType
+        .equals(r.getResourceType()) && (cmp == 0);
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 263167;
+    int result =
+        939769357 + name.hashCode(); // prime * result = 939769357 initially
+    result = prime * result + resourceType.hashCode();
+    result = prime * result + units.hashCode();
+    result = prime * result + value.hashCode();
+    return result;
+  }
+
+  @Override
+  public int compareTo(ResourceInformation other) {
+    int diff = this.name.compareTo(other.name);
+    if (diff == 0) {
+      diff = UnitsConversionUtil
+          .compare(this.units, this.value, other.units, other.value);
+      if (diff == 0) {
+        diff = this.resourceType.compareTo(other.resourceType);
+      }
+    }
+    return diff;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e518ef1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ResourceNotFoundException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ResourceNotFoundException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ResourceNotFoundException.java
new file mode 100644
index 0000000..4277034
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ResourceNotFoundException.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This exception is thrown when details of an unknown resource type
+ * are requested.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ResourceNotFoundException extends YarnException {
+
+  private static final long serialVersionUID = 10081982L;
+
+  public ResourceNotFoundException(String message) {
+    super(message);
+  }
+
+  public ResourceNotFoundException(Throwable cause) {
+    super(cause);
+  }
+
+  public ResourceNotFoundException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e518ef1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/UnitsConversionUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/UnitsConversionUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/UnitsConversionUtil.java
new file mode 100644
index 0000000..7785263
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/UnitsConversionUtil.java
@@ -0,0 +1,197 @@
+/**
+ * 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.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.math.BigInteger;
+import java.util.*;
+
+/**
+ * A util to convert values in one unit to another. Units refers to whether
+ * the value is expressed in pico, nano, etc.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class UnitsConversionUtil {
+
+  /**
+   * Helper class for encapsulating conversion values.
+   */
+  public static class Converter {
+    private long numerator;
+    private long denominator;
+
+    Converter(long n, long d) {
+      this.numerator = n;
+      this.denominator = d;
+    }
+  }
+
+  private static final String[] UNITS =
+      {"p", "n", "u", "m", "", "k", "M", "G", "T", "P"};
+  private static final List<String> SORTED_UNITS = Arrays.asList(UNITS);
+  public static final Set<String> KNOWN_UNITS = createKnownUnitsSet();
+  private static final Converter PICO =
+      new Converter(1L, 1000L * 1000L * 1000L * 1000L);
+  private static final Converter NANO =
+      new Converter(1L, 1000L * 1000L * 1000L);
+  private static final Converter MICRO = new Converter(1L, 1000L * 1000L);
+  private static final Converter MILLI = new Converter(1L, 1000L);
+  private static final Converter BASE = new Converter(1L, 1L);
+  private static final Converter KILO = new Converter(1000L, 1L);
+  private static final Converter MEGA = new Converter(1000L * 1000L, 1L);
+  private static final Converter GIGA =
+      new Converter(1000L * 1000L * 1000L, 1L);
+  private static final Converter TERA =
+      new Converter(1000L * 1000L * 1000L * 1000L, 1L);
+  private static final Converter PETA =
+      new Converter(1000L * 1000L * 1000L * 1000L * 1000L, 1L);
+
+  private static Set<String> createKnownUnitsSet() {
+    Set<String> ret = new HashSet<>();
+    ret.addAll(Arrays.asList(UNITS));
+    return ret;
+  }
+
+  private static Converter getConverter(String unit) {
+    switch (unit) {
+    case "p":
+      return PICO;
+    case "n":
+      return NANO;
+    case "u":
+      return MICRO;
+    case "m":
+      return MILLI;
+    case "":
+      return BASE;
+    case "k":
+      return KILO;
+    case "M":
+      return MEGA;
+    case "G":
+      return GIGA;
+    case "T":
+      return TERA;
+    case "P":
+      return PETA;
+    default:
+      throw new IllegalArgumentException(
+          "Unknown unit '" + unit + "'. Known units are " + KNOWN_UNITS);
+    }
+  }
+
+  /**
+   * Converts a value from one unit to another. Supported units can be obtained
+   * by inspecting the KNOWN_UNITS set.
+   *
+   * @param fromUnit  the unit of the from value
+   * @param toUnit    the target unit
+   * @param fromValue the value you wish to convert
+   * @return the value in toUnit
+   */
+  public static Long convert(String fromUnit, String toUnit, Long fromValue) {
+    if (toUnit == null || fromUnit == null || fromValue == null) {
+      throw new IllegalArgumentException("One or more arguments are null");
+    }
+    Long tmp;
+    String overflowMsg =
+        "Converting " + fromValue + " from '" + fromUnit + "' to '" + toUnit
+            + "' will result in an overflow of Long";
+    Converter fc = getConverter(fromUnit);
+    Converter tc = getConverter(toUnit);
+    Long numerator = fc.numerator * tc.denominator;
+    Long denominator = fc.denominator * tc.numerator;
+    if (numerator < denominator) {
+      if (!toUnit.equals(fromUnit)) {
+        tmp = Long.MAX_VALUE / numerator;
+        if (tmp < fromValue) {
+          throw new IllegalArgumentException(overflowMsg);
+        }
+      }
+      return (fromValue * numerator) / denominator;
+    }
+    tmp = numerator / denominator;
+    if (!toUnit.equals(fromUnit)) {
+      if ((Long.MAX_VALUE / tmp) < fromValue) {
+        throw new IllegalArgumentException(overflowMsg);
+      }
+    }
+    return fromValue * tmp;
+  }
+
+  /**
+   * Compare a value in a given unit with a value in another unit. The return
+   * value is equivalent to the value returned by compareTo.
+   *
+   * @param unitA  first unit
+   * @param valueA first value
+   * @param unitB  second unit
+   * @param valueB second value
+   * @return +1, 0 or -1 depending on whether the relationship is greater than,
+   * equal to or lesser than
+   */
+  public static int compare(String unitA, Long valueA, String unitB,
+      Long valueB) {
+    if (unitA == null || unitB == null || !KNOWN_UNITS.contains(unitA)
+        || !KNOWN_UNITS.contains(unitB)) {
+      throw new IllegalArgumentException("Units cannot be null");
+    }
+    if (!KNOWN_UNITS.contains(unitA)) {
+      throw new IllegalArgumentException("Unknown unit '" + unitA + "'");
+    }
+    if (!KNOWN_UNITS.contains(unitB)) {
+      throw new IllegalArgumentException("Unknown unit '" + unitB + "'");
+    }
+    Converter unitAC = getConverter(unitA);
+    Converter unitBC = getConverter(unitB);
+    if (unitA.equals(unitB)) {
+      return valueA.compareTo(valueB);
+    }
+    int unitAPos = SORTED_UNITS.indexOf(unitA);
+    int unitBPos = SORTED_UNITS.indexOf(unitB);
+    try {
+      Long tmpA = valueA;
+      Long tmpB = valueB;
+      if (unitAPos < unitBPos) {
+        tmpB = convert(unitB, unitA, valueB);
+      } else {
+        tmpA = convert(unitA, unitB, valueA);
+      }
+      return tmpA.compareTo(tmpB);
+    } catch (IllegalArgumentException ie) {
+      BigInteger tmpA = BigInteger.valueOf(valueA);
+      BigInteger tmpB = BigInteger.valueOf(valueB);
+      if (unitAPos < unitBPos) {
+        tmpB = tmpB.multiply(BigInteger.valueOf(unitBC.numerator));
+        tmpB = tmpB.multiply(BigInteger.valueOf(unitAC.denominator));
+        tmpB = tmpB.divide(BigInteger.valueOf(unitBC.denominator));
+        tmpB = tmpB.divide(BigInteger.valueOf(unitAC.numerator));
+      } else {
+        tmpA = tmpA.multiply(BigInteger.valueOf(unitAC.numerator));
+        tmpA = tmpA.multiply(BigInteger.valueOf(unitBC.denominator));
+        tmpA = tmpA.divide(BigInteger.valueOf(unitAC.denominator));
+        tmpA = tmpA.divide(BigInteger.valueOf(unitBC.numerator));
+      }
+      return tmpA.compareTo(tmpB);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e518ef1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 0649f8e..cf8f760 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -53,9 +53,21 @@ message ContainerIdProto {
   optional int64 id = 3;
 }
 
+enum ResourceTypesProto {
+  COUNTABLE = 0;
+}
+
+message ResourceInformationProto {
+  required string key = 1;
+  optional int64 value = 2;
+  optional string units = 3;
+  optional ResourceTypesProto type = 4;
+}
+
 message ResourceProto {
   optional int64 memory = 1;
   optional int64 virtual_cores = 2;
+  repeated ResourceInformationProto resource_value_map = 3;
 }
 
 message ResourceUtilizationProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e518ef1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestResourceInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestResourceInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestResourceInformation.java
new file mode 100644
index 0000000..28f69c9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestResourceInformation.java
@@ -0,0 +1,70 @@
+/**
+ * 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.conf;
+
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestResourceInformation {
+
+  @Test
+  public void testName() {
+    String name = "yarn.io/test";
+    ResourceInformation ri = ResourceInformation.newInstance(name);
+    Assert.assertEquals("Resource name incorrect", name, ri.getName());
+  }
+
+  @Test
+  public void testUnits() {
+    String name = "yarn.io/test";
+    String units = "m";
+    ResourceInformation ri = ResourceInformation.newInstance(name, units);
+    Assert.assertEquals("Resource name incorrect", name, ri.getName());
+    Assert.assertEquals("Resource units incorrect", units, ri.getUnits());
+    units = "z";
+    try {
+      ResourceInformation.newInstance(name, units);
+      Assert.fail(units + "is not a valid unit");
+    } catch (IllegalArgumentException ie) {
+      // do nothing
+    }
+  }
+
+  @Test
+  public void testValue() {
+    String name = "yarn.io/test";
+    Long value = 1l;
+    ResourceInformation ri = ResourceInformation.newInstance(name, value);
+    Assert.assertEquals("Resource name incorrect", name, ri.getName());
+    Assert.assertEquals("Resource value incorrect", value, ri.getValue());
+  }
+
+  @Test
+  public void testResourceInformation() {
+    String name = "yarn.io/test";
+    Long value = 1l;
+    String units = "m";
+    ResourceInformation ri =
+        ResourceInformation.newInstance(name, units, value);
+    Assert.assertEquals("Resource name incorrect", name, ri.getName());
+    Assert.assertEquals("Resource value incorrect", value, ri.getValue());
+    Assert.assertEquals("Resource units incorrect", units, ri.getUnits());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e518ef1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/util/TestUnitsConversionUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/util/TestUnitsConversionUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/util/TestUnitsConversionUtil.java
new file mode 100644
index 0000000..421768f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/util/TestUnitsConversionUtil.java
@@ -0,0 +1,120 @@
+/**
+ * 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.util;
+
+import org.apache.hadoop.yarn.util.UnitsConversionUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestUnitsConversionUtil {
+
+  @Test
+  public void testUnitsConversion() {
+    int value = 5;
+    String fromUnit = "";
+    Long test = Long.valueOf(value);
+    Assert.assertEquals("pico test failed",
+        Long.valueOf(value * 1000l * 1000l * 1000l * 1000l),
+        UnitsConversionUtil.convert(fromUnit, "p", test));
+    Assert.assertEquals("nano test failed",
+        Long.valueOf(value * 1000l * 1000l * 1000l),
+        UnitsConversionUtil.convert(fromUnit, "n", test));
+    Assert
+        .assertEquals("micro test failed", Long.valueOf(value * 1000l * 1000l),
+            UnitsConversionUtil.convert(fromUnit, "u", test));
+    Assert.assertEquals("milli test failed", Long.valueOf(value * 1000l),
+        UnitsConversionUtil.convert(fromUnit, "m", test));
+
+    test = Long.valueOf(value * 1000l * 1000l * 1000l * 1000l * 1000l);
+    fromUnit = "";
+    Assert.assertEquals("kilo test failed", Long.valueOf(test / 1000l),
+        UnitsConversionUtil.convert(fromUnit, "k", test));
+    Assert
+        .assertEquals("mega test failed", Long.valueOf(test / (1000l * 1000l)),
+            UnitsConversionUtil.convert(fromUnit, "M", test));
+    Assert.assertEquals("giga test failed",
+        Long.valueOf(test / (1000l * 1000l * 1000l)),
+        UnitsConversionUtil.convert(fromUnit, "G", test));
+    Assert.assertEquals("tera test failed",
+        Long.valueOf(test / (1000l * 1000l * 1000l * 1000l)),
+        UnitsConversionUtil.convert(fromUnit, "T", test));
+    Assert.assertEquals("peta test failed",
+        Long.valueOf(test / (1000l * 1000l * 1000l * 1000l * 1000l)),
+        UnitsConversionUtil.convert(fromUnit, "P", test));
+
+    Assert.assertEquals("nano to pico test failed", Long.valueOf(value * 1000l),
+        UnitsConversionUtil.convert("n", "p", Long.valueOf(value)));
+
+    Assert.assertEquals("mega to giga test failed", Long.valueOf(value),
+        UnitsConversionUtil.convert("M", "G", Long.valueOf(value * 1000l)));
+  }
+
+  @Test
+  public void testOverflow() {
+    Long test = Long.valueOf(5 * 1000l * 1000l * 1000l * 1000l * 1000l);
+    try {
+      UnitsConversionUtil.convert("P", "p", test);
+      Assert.fail("this operation should result in an overflow");
+    } catch (IllegalArgumentException ie) {
+      ; // do nothing
+    }
+    try {
+      UnitsConversionUtil.convert("m", "p", Long.MAX_VALUE - 1);
+      Assert.fail("this operation should result in an overflow");
+    } catch (IllegalArgumentException ie) {
+      ; // do nothing
+    }
+  }
+
+  @Test
+  public void testCompare() {
+    String unitA = "P";
+    Long valueA = Long.valueOf(1);
+    String unitB = "p";
+    Long valueB = Long.valueOf(2);
+    Assert.assertEquals(1,
+        UnitsConversionUtil.compare(unitA, valueA, unitB, valueB));
+    Assert.assertEquals(-1,
+        UnitsConversionUtil.compare(unitB, valueB, unitA, valueA));
+    Assert.assertEquals(0,
+        UnitsConversionUtil.compare(unitA, valueA, unitA, valueA));
+    Assert.assertEquals(-1,
+        UnitsConversionUtil.compare(unitA, valueA, unitA, valueB));
+    Assert.assertEquals(1,
+        UnitsConversionUtil.compare(unitA, valueB, unitA, valueA));
+
+    unitB = "T";
+    Assert.assertEquals(1,
+        UnitsConversionUtil.compare(unitA, valueA, unitB, valueB));
+    Assert.assertEquals(-1,
+        UnitsConversionUtil.compare(unitB, valueB, unitA, valueA));
+    Assert.assertEquals(0,
+        UnitsConversionUtil.compare(unitA, valueA, unitB, 1000l));
+
+    unitA = "p";
+    unitB = "n";
+    Assert.assertEquals(-1,
+        UnitsConversionUtil.compare(unitA, valueA, unitB, valueB));
+    Assert.assertEquals(1,
+        UnitsConversionUtil.compare(unitB, valueB, unitA, valueA));
+    Assert.assertEquals(0,
+        UnitsConversionUtil.compare(unitA, 1000l, unitB, valueA));
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e518ef1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
index 1a0f30a..ee2f211 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
@@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.protocolrecords.ApplicationsRequestScope;
+import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
 import org.apache.hadoop.yarn.api.records.AMCommand;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
@@ -62,6 +63,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ContainerRetryPolicyProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerTypeProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ExecutionTypeProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ExecutionTypeRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceTypesProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos;
 import org.apache.hadoop.yarn.server.api.ContainerType;
 
@@ -339,4 +341,15 @@ public class ProtoUtils {
       ExecutionTypeRequestProto e) {
     return new ExecutionTypeRequestPBImpl(e);
   }
+
+  /*
+   * ResourceTypes
+   */
+  public static ResourceTypesProto converToProtoFormat(ResourceTypes e) {
+    return ResourceTypesProto.valueOf(e.name());
+  }
+
+  public static ResourceTypes convertFromProtoFormat(ResourceTypesProto e) {
+    return ResourceTypes.valueOf(e.name());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e518ef1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
index 937941a..dc5c702 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
@@ -21,9 +21,17 @@ package org.apache.hadoop.yarn.api.records.impl.pb;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
+import org.apache.hadoop.yarn.exceptions.ResourceNotFoundException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceInformationProto;
+import org.apache.hadoop.yarn.util.UnitsConversionUtil;
+
+import java.util.*;
 
 @Private
 @Unstable
@@ -31,7 +39,9 @@ public class ResourcePBImpl extends Resource {
   ResourceProto proto = ResourceProto.getDefaultInstance();
   ResourceProto.Builder builder = null;
   boolean viaProto = false;
-  
+
+  private Map<String, ResourceInformation> resources;
+
   public ResourcePBImpl() {
     builder = ResourceProto.newBuilder();
   }
@@ -39,9 +49,12 @@ public class ResourcePBImpl extends Resource {
   public ResourcePBImpl(ResourceProto proto) {
     this.proto = proto;
     viaProto = true;
+    this.resources = null;
+    initResources();
   }
-  
+
   public ResourceProto getProto() {
+    mergeLocalToProto();
     proto = viaProto ? proto : builder.build();
     viaProto = true;
     return proto;
@@ -57,19 +70,29 @@ public class ResourcePBImpl extends Resource {
   @Override
   @SuppressWarnings("deprecation")
   public int getMemory() {
-    return (int) getMemorySize();
+    return (int) this.getMemorySize();
   }
 
   @Override
   public long getMemorySize() {
-    ResourceProtoOrBuilder p = viaProto ? proto : builder;
-    return p.getMemory();
+    try {
+      ResourceInformation ri =
+          this.getResourceInformation(ResourceInformation.MEMORY.getName());
+      return (int) UnitsConversionUtil
+          .convert(ri.getUnits(), "M", ri.getValue()).longValue();
+    } catch (YarnException ye) {
+      // memory should always be present
+      initResourcesMap();
+      return 0;
+    }
   }
 
   @Override
   public void setMemory(long memory) {
-    maybeInitBuilder();
-    builder.setMemory(memory);
+    setResourceInformation(ResourceInformation.MEMORY_MB.getName(),
+        ResourceInformation.newInstance(ResourceInformation.MEMORY_MB.getName(),
+            ResourceInformation.MEMORY_MB.getUnits(), (long) memory));
+
   }
 
   @Override
@@ -79,24 +102,164 @@ public class ResourcePBImpl extends Resource {
 
   @Override
   public long getVirtualCoresSize() {
-    ResourceProtoOrBuilder p = viaProto ? proto : builder;
-    return p.getVirtualCores();
+    try {
+      return (int) this.getResourceValue(ResourceInformation.VCORES.getName())
+          .longValue();
+    } catch (YarnException ye) {
+      // vcores should always be present
+      initResourcesMap();
+      return 0;
+    }
   }
 
   @Override
   public void setVirtualCores(long vCores) {
+    try {
+      setResourceValue(ResourceInformation.VCORES.getName(),
+          Long.valueOf(vCores));
+    } catch (ResourceNotFoundException re) {
+      this.setResourceInformation(ResourceInformation.VCORES.getName(),
+          ResourceInformation.newInstance(ResourceInformation.VCORES.getName(),
+              (long) vCores));
+    }
+  }
+
+  private void initResources() {
+    if (this.resources != null) {
+      return;
+    }
+    ResourceProtoOrBuilder p = viaProto ? proto : builder;
+    initResourcesMap();
+    for (ResourceInformationProto entry : p.getResourceValueMapList()) {
+      ResourceTypes type =
+          entry.hasType() ? ProtoUtils.convertFromProtoFormat(entry.getType()) :
+              ResourceTypes.COUNTABLE;
+      String units = entry.hasUnits() ? entry.getUnits() : "";
+      Long value = entry.hasValue() ? entry.getValue() : 0L;
+      ResourceInformation ri =
+          ResourceInformation.newInstance(entry.getKey(), units, value, type);
+      resources.put(ri.getName(), ri);
+    }
+    if(this.getMemory() != p.getMemory()) {
+      setMemory(p.getMemory());
+    }
+    if(this.getVirtualCores() != p.getVirtualCores()) {
+      setVirtualCores(p.getVirtualCores());
+    }
+  }
+
+  @Override
+  public void setResources(Map<String, ResourceInformation> resources) {
     maybeInitBuilder();
-    builder.setVirtualCores(vCores);
+    if (resources == null || resources.isEmpty()) {
+      builder.clearResourceValueMap();
+    } else {
+      for (Map.Entry<String, ResourceInformation> entry : resources.entrySet()) {
+        if (!entry.getKey().equals(entry.getValue().getName())) {
+          entry.getValue().setName(entry.getKey());
+        }
+      }
+    }
+    this.resources = resources;
   }
 
   @Override
-  public int compareTo(Resource other) {
-    long diff = this.getMemorySize() - other.getMemorySize();
-    if (diff == 0) {
-      diff = this.getVirtualCores() - other.getVirtualCores();
+  public void setResourceInformation(String resource,
+      ResourceInformation resourceInformation) {
+    maybeInitBuilder();
+    if (resource == null || resourceInformation == null) {
+      throw new IllegalArgumentException(
+          "resource and/or resourceInformation cannot be null");
+    }
+    if (!resource.equals(resourceInformation.getName())) {
+      resourceInformation.setName(resource);
     }
-    return diff == 0 ? 0 : (diff > 0 ? 1 : -1);
+    initResourcesMap();
+    resources.put(resource, resourceInformation);
+  }
+
+  @Override
+  public void setResourceValue(String resource, Long value)
+      throws ResourceNotFoundException {
+    maybeInitBuilder();
+    if (resource == null) {
+      throw new IllegalArgumentException("resource type object cannot be null");
+    }
+    if (resources == null || (!resources.containsKey(resource))) {
+      throw new ResourceNotFoundException(
+          "Resource " + resource + " not found");
+    }
+    ResourceInformation ri = resources.get(resource);
+    ri.setValue(value);
+    resources.put(resource, ri);
+  }
+
+  @Override
+  public Map<String, ResourceInformation> getResources() {
+    initResources();
+    return Collections.unmodifiableMap(this.resources);
+  }
+
+  @Override
+  public ResourceInformation getResourceInformation(String resource)
+      throws YarnException {
+    initResources();
+    if (this.resources.containsKey(resource)) {
+      return this.resources.get(resource);
+    }
+    throw new ResourceNotFoundException("Could not find entry for " + resource);
+  }
+
+  @Override
+  public Long getResourceValue(String resource) throws YarnException {
+    initResources();
+    if (this.resources.containsKey(resource)) {
+      return this.resources.get(resource).getValue();
+    }
+    throw new ResourceNotFoundException("Could not find entry for " + resource);
+  }
+
+  private void initResourcesMap() {
+    if (resources == null) {
+      resources = new HashMap<>();
+    }
+    ResourceInformation ri;
+    if (!resources.containsKey(ResourceInformation.MEMORY.getName())) {
+      ri = ResourceInformation
+          .newInstance(ResourceInformation.MEMORY_MB.getName(),
+              ResourceInformation.MEMORY_MB.getUnits());
+      this.resources.put(ResourceInformation.MEMORY.getName(), ri);
+    }
+    if (!resources.containsKey(ResourceInformation.VCORES.getName())) {
+      ri =
+          ResourceInformation.newInstance(ResourceInformation.VCORES.getName());
+      this.resources.put(ResourceInformation.VCORES.getName(), ri);
+    }
+  }
+
+  synchronized private void mergeLocalToBuilder() {
+    builder.clearResourceValueMap();
+    if (resources != null && !resources.isEmpty()) {
+      for (Map.Entry<String, ResourceInformation> entry : resources.entrySet()) {
+        ResourceInformationProto.Builder e = ResourceInformationProto.newBuilder();
+        e.setKey(entry.getKey());
+        e.setUnits(entry.getValue().getUnits());
+        e.setType(
+            ProtoUtils.converToProtoFormat(entry.getValue().getResourceType()));
+        e.setValue(entry.getValue().getValue());
+        builder.addResourceValueMap(e);
+      }
+    }
+    builder.setMemory(this.getMemory());
+    builder.setVirtualCores(this.getVirtualCores());
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
   }
-  
-  
 }  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e518ef1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
index e944e11..c9ff64f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
@@ -21,100 +21,141 @@ package org.apache.hadoop.yarn.util.resource;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
+import org.apache.hadoop.yarn.exceptions.ResourceNotFoundException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.Records;
 
-@InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+@InterfaceAudience.LimitedPrivate({ "YARN", "MapReduce" })
 @Unstable
 public class Resources {
-  
-  // Java doesn't have const :(
-  private static final Resource NONE = new Resource() {
+
+  /**
+   * Helper class to create a resource with a fixed value for all resource
+   * types. For example, a NONE resource which returns 0 for any resource type.
+   */
+  static class FixedValueResource extends Resource {
+
+    private Map<String, ResourceInformation> resources;
+    private Long resourceValue;
+    private String name;
+
+    /**
+     * Constructor for a fixed value resource
+     * @param rName the name of the resource
+     * @param value the fixed value to be returned for all resource types
+     */
+    FixedValueResource(String rName, Long value) {
+      this.resourceValue = value;
+      this.name = rName;
+      resources = initResourceMap();
+    }
+
+    private int resourceValueToInt() {
+      if(this.resourceValue > Integer.MAX_VALUE) {
+        return Integer.MAX_VALUE;
+      }
+      return this.resourceValue.intValue();
+    }
 
     @Override
     @SuppressWarnings("deprecation")
     public int getMemory() {
-      return 0;
+      return resourceValueToInt();
     }
 
     @Override
     public long getMemorySize() {
-      return 0;
+      return this.resourceValue;
     }
 
     @Override
     public void setMemory(long memory) {
-      throw new RuntimeException("NONE cannot be modified!");
+      throw new RuntimeException(name + " cannot be modified!");
     }
 
     @Override
     public int getVirtualCores() {
-      return 0;
+      return resourceValueToInt();
     }
 
     @Override
     public long getVirtualCoresSize() {
-      return 0;
+      return this.resourceValue;
     }
 
     @Override
-    public void setVirtualCores(long cores) {
-      throw new RuntimeException("NONE cannot be modified!");
+    public Map<String, ResourceInformation> getResources() {
+      return Collections.unmodifiableMap(this.resources);
     }
 
     @Override
-    public int compareTo(Resource o) {
-      long diff = 0 - o.getMemorySize();
-      if (diff == 0) {
-        diff = 0 - o.getVirtualCores();
+    public ResourceInformation getResourceInformation(String resource)
+        throws YarnException {
+      if (resources.containsKey(resource)) {
+        ResourceInformation value = this.resources.get(resource);
+        ResourceInformation ret = ResourceInformation.newInstance(value);
+        ret.setValue(resourceValue);
+        return ret;
       }
-      return Long.signum(diff);
+      throw new YarnException("" + resource + " not found");
     }
-    
-  };
-  
-  private static final Resource UNBOUNDED = new Resource() {
 
     @Override
-    @SuppressWarnings("deprecation")
-    public int getMemory() {
-      return Integer.MAX_VALUE;
+    public Long getResourceValue(String resource) throws YarnException {
+      if (resources.containsKey(resource)) {
+        return resourceValue;
+      }
+      throw new YarnException("" + resource + " not found");
     }
 
     @Override
-    public long getMemorySize() {
-      return Long.MAX_VALUE;
+    public void setVirtualCores(long cores) {
+      throw new RuntimeException(name + " cannot be modified!");
     }
 
     @Override
-    public void setMemory(long memory) {
-      throw new RuntimeException("UNBOUNDED cannot be modified!");
+    public void setResources(Map<String, ResourceInformation> resources) {
+      throw new RuntimeException(name + " cannot be modified!");
     }
 
     @Override
-    public int getVirtualCores() {
-      return Integer.MAX_VALUE;
+    public void setResourceInformation(String resource,
+        ResourceInformation resourceInformation)
+        throws ResourceNotFoundException {
+      throw new RuntimeException(name + " cannot be modified!");
     }
 
     @Override
-    public long getVirtualCoresSize() {
-      return Long.MAX_VALUE;
+    public void setResourceValue(String resource, Long value)
+        throws ResourceNotFoundException {
+      throw new RuntimeException(name + " cannot be modified!");
     }
 
-    @Override
-    public void setVirtualCores(long cores) {
-      throw new RuntimeException("UNBOUNDED cannot be modified!");
+    private Map<String, ResourceInformation> initResourceMap() {
+      Map<String, ResourceInformation> tmp = new HashMap<>();
+      // Due to backwards compat, the max value for memory and vcores
+      // needs to be Integer.MAX_VALUE
+      int max = resourceValue > Integer.MAX_VALUE ? Integer.MAX_VALUE :
+          resourceValue.intValue();
+      tmp.put(ResourceInformation.MEMORY.getName(), ResourceInformation
+          .newInstance(ResourceInformation.MEMORY.getName(),
+              ResourceInformation.MEMORY_MB.getUnits(), (long) max));
+      tmp.put(ResourceInformation.VCORES.getName(), ResourceInformation
+          .newInstance(ResourceInformation.VCORES.getName(), (long) max));
+      return tmp;
     }
 
-    @Override
-    public int compareTo(Resource o) {
-      long diff = Long.MAX_VALUE - o.getMemorySize();
-      if (diff == 0) {
-        diff = Long.MAX_VALUE - o.getVirtualCoresSize();
-      }
-      return Long.signum(diff);
-    }
-    
-  };
+  }
+
+  private static final Resource UNBOUNDED =
+      new FixedValueResource("UNBOUNDED", Long.MAX_VALUE);
+
+  private static final Resource NONE = new FixedValueResource("NONE", 0L);
 
   public static Resource createResource(long memory) {
     return createResource(memory, (memory > 0) ? 1 : 0);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e518ef1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index 55b1233..6637050 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -145,6 +145,7 @@ import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.ReservationRequests;
 import org.apache.hadoop.yarn.api.records.ResourceAllocationRequest;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -462,6 +463,8 @@ public class TestPBImplRecords {
     typeValueCache.put(SerializedException.class,
         SerializedException.newInstance(new IOException("exception for test")));
     generateByNewInstance(ExecutionTypeRequest.class);
+    typeValueCache.put(ResourceInformation.class, ResourceInformation
+        .newInstance("localhost.test/sample", 1l));
     generateByNewInstance(LogAggregationContext.class);
     generateByNewInstance(ApplicationId.class);
     generateByNewInstance(ApplicationAttemptId.class);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[28/51] [abbrv] hadoop git commit: YARN-3426. Add jdiff support to YARN. (vinodkv via wangda)

Posted by vv...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/03fc6b1b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_2.6.0.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_2.6.0.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_2.6.0.xml
new file mode 100644
index 0000000..094962f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_2.6.0.xml
@@ -0,0 +1,2059 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!--
+   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.
+-->
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Wed Apr 08 11:30:15 PDT 2015 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="hadoop-yarn-server-common 2.6.0"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.ExcludePrivateAnnotationsJDiffDoclet -docletpath /Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/target/hadoop-annotations.jar:/Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/target/jdiff.jar -verbose -classpath /Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/target/classes:/Users/llu/.m2/repository/org/apache/hadoop/hadoop-common/2.6.0/hadoop-common-2.6.0.jar:/Users/llu/.m2/repository/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/Users/llu/.m2/repository/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/Users/llu/.m2/repository/xmlenc/xmlenc/0.52/xmlenc-0.52.jar:/Users/llu/.m2/repository/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1.jar:/Users/llu/.m2/repository/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/Users
 /llu/.m2/repository/commons-io/commons-io/2.4/commons-io-2.4.jar:/Users/llu/.m2/repository/commons-net/commons-net/3.1/commons-net-3.1.jar:/Users/llu/.m2/repository/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1.jar:/Users/llu/.m2/repository/javax/servlet/servlet-api/2.5/servlet-api-2.5.jar:/Users/llu/.m2/repository/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/Users/llu/.m2/repository/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/Users/llu/.m2/repository/com/sun/jersey/jersey-core/1.9/jersey-core-1.9.jar:/Users/llu/.m2/repository/com/sun/jersey/jersey-json/1.9/jersey-json-1.9.jar:/Users/llu/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/Users/llu/.m2/repository/com/sun/xml/bind/jaxb-impl/2.2.3-1/jaxb-impl-2.2.3-1.jar:/Users/llu/.m2/repository/com/sun/jersey/jersey-server/1.9/jersey-server-1.9.jar:/Users/llu/.m2/repository/asm/asm/3.2/asm-3.2.jar:/Users/llu/.m2/repository/javax/servlet/jsp/jsp-api/2.1/jsp-api-2.1.jar:/Use
 rs/llu/.m2/repository/log4j/log4j/1.2.17/log4j-1.2.17.jar:/Users/llu/.m2/repository/net/java/dev/jets3t/jets3t/0.9.0/jets3t-0.9.0.jar:/Users/llu/.m2/repository/org/apache/httpcomponents/httpclient/4.2.5/httpclient-4.2.5.jar:/Users/llu/.m2/repository/org/apache/httpcomponents/httpcore/4.2.5/httpcore-4.2.5.jar:/Users/llu/.m2/repository/com/jamesmurty/utils/java-xmlbuilder/0.4/java-xmlbuilder-0.4.jar:/Users/llu/.m2/repository/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/Users/llu/.m2/repository/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/Users/llu/.m2/repository/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/Users/llu/.m2/repository/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/Users/llu/.m2/repository/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar:/Users/llu/.m2/repository/org/slf4j/slf4j-api/1.7.5/slf4j-api-1.7.5.jar:/Users/llu/.m2/repository/org/slf4j/slf4j-log4j12/1.
 7.5/slf4j-log4j12-1.7.5.jar:/Users/llu/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/llu/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/llu/.m2/repository/org/apache/avro/avro/1.7.4/avro-1.7.4.jar:/Users/llu/.m2/repository/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/Users/llu/.m2/repository/org/xerial/snappy/snappy-java/1.0.4.1/snappy-java-1.0.4.1.jar:/Users/llu/.m2/repository/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/Users/llu/.m2/repository/org/apache/hadoop/hadoop-auth/2.6.0/hadoop-auth-2.6.0.jar:/Users/llu/.m2/repository/org/apache/directory/server/apacheds-kerberos-codec/2.0.0-M15/apacheds-kerberos-codec-2.0.0-M15.jar:/Users/llu/.m2/repository/org/apache/directory/server/apacheds-i18n/2.0.0-M15/apacheds-i18n-2.0.0-M15.jar:/Users/llu/.m2/repository/org/apache/directory/api/api-asn1-api/1.0.0-M20/api-asn1-api-1.0.0-M20.jar:/Users/llu/.m2/repository/org/apache
 /directory/api/api-util/1.0.0-M20/api-util-1.0.0-M20.jar:/Users/llu/.m2/repository/org/apache/curator/curator-framework/2.6.0/curator-framework-2.6.0.jar:/Users/llu/.m2/repository/com/jcraft/jsch/0.1.42/jsch-0.1.42.jar:/Users/llu/.m2/repository/org/apache/curator/curator-client/2.6.0/curator-client-2.6.0.jar:/Users/llu/.m2/repository/org/apache/curator/curator-recipes/2.6.0/curator-recipes-2.6.0.jar:/Users/llu/.m2/repository/com/google/code/findbugs/jsr305/1.3.9/jsr305-1.3.9.jar:/Users/llu/.m2/repository/org/htrace/htrace-core/3.0.4/htrace-core-3.0.4.jar:/Users/llu/.m2/repository/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/Users/llu/.m2/repository/org/tukaani/xz/1.0/xz-1.0.jar:/Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/hadoop-yarn-api-2.6.0.jar:/Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/hadoop-yarn-common-2.6.0.jar:/Users/llu/.m2/repository/javax/xml/bind/jaxb-api/2.2.2/jaxb-api-2.2.2.jar:
 /Users/llu/.m2/repository/javax/xml/stream/stax-api/1.0-2/stax-api-1.0-2.jar:/Users/llu/.m2/repository/javax/activation/activation/1.1/activation-1.1.jar:/Users/llu/.m2/repository/com/sun/jersey/jersey-client/1.9/jersey-client-1.9.jar:/Users/llu/.m2/repository/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/Users/llu/.m2/repository/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/Users/llu/.m2/repository/com/google/inject/extensions/guice-servlet/3.0/guice-servlet-3.0.jar:/Users/llu/.m2/repository/com/google/inject/guice/3.0/guice-3.0.jar:/Users/llu/.m2/repository/javax/inject/javax.inject/1/javax.inject-1.jar:/Users/llu/.m2/repository/aopalliance/aopalliance/1.0/aopalliance-1.0.jar:/Users/llu/.m2/repository/com/sun/jersey/contribs/jersey-guice/1.9/jersey-guice-1.9.jar:/Users/llu/.m2/repository/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/Users/llu/.m2/repository/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/Users/llu/.m2/reposi
 tory/org/apache/hadoop/hadoop-annotations/2.6.0/hadoop-annotations-2.6.0.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/lib/tools.jar:/Users/llu/.m2/repository/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar:/Users/llu/.m2/repository/org/apache/zookeeper/zookeeper/3.4.6/zookeeper-3.4.6.jar:/Users/llu/.m2/repository/jline/jline/0.9.94/jline-0.9.94.jar:/Users/llu/.m2/repository/io/netty/netty/3.6.2.Final/netty-3.6.2.Final.jar:/Users/llu/.m2/repository/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar -sourcepath /Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java -apidir /Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/target/site/jdiff/xml -apiname hadoop-yarn-server-common 2.6.0 -->
+<package name="org.apache.hadoop.yarn.server">
+  <!-- start class org.apache.hadoop.yarn.server.RMNMSecurityInfoClass -->
+  <class name="RMNMSecurityInfoClass" extends="org.apache.hadoop.security.SecurityInfo"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="RMNMSecurityInfoClass"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getKerberosInfo" return="org.apache.hadoop.security.KerberosInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getTokenInfo" return="org.apache.hadoop.security.token.TokenInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.RMNMSecurityInfoClass -->
+</package>
+<package name="org.apache.hadoop.yarn.server.api">
+  <!-- start interface org.apache.hadoop.yarn.server.api.ApplicationContext -->
+  <interface name="ApplicationContext"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getApplication" return="org.apache.hadoop.yarn.api.records.ApplicationReport"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[This method returns Application {@link ApplicationReport} for the specified
+ {@link ApplicationId}.
+
+ @param appId
+
+ @return {@link ApplicationReport} for the ApplicationId.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getAllApplications" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[This method returns all Application {@link ApplicationReport}s
+
+ @return map of {@link ApplicationId} to {@link ApplicationReport}s.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttempts" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Application can have multiple application attempts
+ {@link ApplicationAttemptReport}. This method returns the all
+ {@link ApplicationAttemptReport}s for the Application.
+
+ @param appId
+
+ @return all {@link ApplicationAttemptReport}s for the Application.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttempt" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptReport"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[This method returns {@link ApplicationAttemptReport} for specified
+ {@link ApplicationId}.
+
+ @param appAttemptId
+          {@link ApplicationAttemptId}
+ @return {@link ApplicationAttemptReport} for ApplicationAttemptId
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainer" return="org.apache.hadoop.yarn.api.records.ContainerReport"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[This method returns {@link ContainerReport} for specified
+ {@link ContainerId}.
+
+ @param containerId
+          {@link ContainerId}
+ @return {@link ContainerReport} for ContainerId
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getAMContainer" return="org.apache.hadoop.yarn.api.records.ContainerReport"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[This method returns {@link ContainerReport} for specified
+ {@link ApplicationAttemptId}.
+
+ @param appAttemptId
+          {@link ApplicationAttemptId}
+ @return {@link ContainerReport} for ApplicationAttemptId
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainers" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[This method returns Map of {@link ContainerId} to {@link ContainerReport}
+ for specified {@link ApplicationAttemptId}.
+
+ @param appAttemptId
+          {@link ApplicationAttemptId}
+ @return Map of {@link ContainerId} to {@link ContainerReport} for
+         ApplicationAttemptId
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.server.api.ApplicationContext -->
+  <!-- start interface org.apache.hadoop.yarn.server.api.ResourceManagerConstants -->
+  <interface name="ResourceManagerConstants"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <field name="RM_INVALID_IDENTIFIER" type="long"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[This states the invalid identifier of Resource Manager. This is used as a
+ default value for initializing RM identifier. Currently, RM is using time
+ stamp as RM identifier.]]>
+      </doc>
+    </field>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.server.api.ResourceManagerConstants -->
+  <!-- start interface org.apache.hadoop.yarn.server.api.ResourceTracker -->
+  <interface name="ResourceTracker"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="registerNodeManager" return="org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="nodeHeartbeat" return="org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.server.api.ResourceTracker -->
+  <!-- start interface org.apache.hadoop.yarn.server.api.ResourceTrackerPB -->
+  <interface name="ResourceTrackerPB"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.proto.ResourceTracker.ResourceTrackerService.BlockingInterface"/>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.server.api.ResourceTrackerPB -->
+  <!-- start class org.apache.hadoop.yarn.server.api.ServerRMProxy -->
+  <class name="ServerRMProxy" extends="org.apache.hadoop.yarn.client.RMProxy"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="createRMProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="configuration" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Create a proxy to the ResourceManager for the specified protocol.
+ @param configuration Configuration with all the required information.
+ @param protocol Server protocol for which proxy is being requested.
+ @param <T> Type of proxy.
+ @return Proxy to the ResourceManager for the specified server protocol.
+ @throws IOException]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.ServerRMProxy -->
+</package>
+<package name="org.apache.hadoop.yarn.server.api.impl.pb.client">
+  <!-- start class org.apache.hadoop.yarn.server.api.impl.pb.client.ResourceTrackerPBClientImpl -->
+  <class name="ResourceTrackerPBClientImpl" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.server.api.ResourceTracker"/>
+    <implements name="java.io.Closeable"/>
+    <constructor name="ResourceTrackerPBClientImpl" type="long, java.net.InetSocketAddress, org.apache.hadoop.conf.Configuration"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </constructor>
+    <method name="close"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="registerNodeManager" return="org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="nodeHeartbeat" return="org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.impl.pb.client.ResourceTrackerPBClientImpl -->
+</package>
+<package name="org.apache.hadoop.yarn.server.api.impl.pb.service">
+  <!-- start class org.apache.hadoop.yarn.server.api.impl.pb.service.ResourceTrackerPBServiceImpl -->
+  <class name="ResourceTrackerPBServiceImpl" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.server.api.ResourceTrackerPB"/>
+    <constructor name="ResourceTrackerPBServiceImpl" type="org.apache.hadoop.yarn.server.api.ResourceTracker"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="registerNodeManager" return="org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.RegisterNodeManagerResponseProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="controller" type="com.google.protobuf.RpcController"/>
+      <param name="proto" type="org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.RegisterNodeManagerRequestProto"/>
+      <exception name="ServiceException" type="com.google.protobuf.ServiceException"/>
+    </method>
+    <method name="nodeHeartbeat" return="org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="controller" type="com.google.protobuf.RpcController"/>
+      <param name="proto" type="org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProto"/>
+      <exception name="ServiceException" type="com.google.protobuf.ServiceException"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.impl.pb.service.ResourceTrackerPBServiceImpl -->
+</package>
+<package name="org.apache.hadoop.yarn.server.api.records">
+  <!-- start interface org.apache.hadoop.yarn.server.api.records.MasterKey -->
+  <interface name="MasterKey"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getKeyId" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setKeyId"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="keyId" type="int"/>
+    </method>
+    <method name="getBytes" return="java.nio.ByteBuffer"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setBytes"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="bytes" type="java.nio.ByteBuffer"/>
+    </method>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.server.api.records.MasterKey -->
+  <!-- start class org.apache.hadoop.yarn.server.api.records.NodeAction -->
+  <class name="NodeAction" extends="java.lang.Enum"
+    abstract="false"
+    static="false" final="true" visibility="public"
+    deprecated="not deprecated">
+    <method name="values" return="org.apache.hadoop.yarn.server.api.records.NodeAction[]"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="valueOf" return="org.apache.hadoop.yarn.server.api.records.NodeAction"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+    </method>
+    <doc>
+    <![CDATA[The NodeManager is instructed to perform the given action.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.records.NodeAction -->
+  <!-- start class org.apache.hadoop.yarn.server.api.records.NodeHealthStatus -->
+  <class name="NodeHealthStatus" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NodeHealthStatus"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getIsNodeHealthy" return="boolean"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Is the node healthy?
+ @return <code>true</code> if the node is healthy, else <code>false</code>]]>
+      </doc>
+    </method>
+    <method name="getHealthReport" return="java.lang.String"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <em>diagnostic health report</em> of the node.
+ @return <em>diagnostic health report</em> of the node]]>
+      </doc>
+    </method>
+    <method name="getLastHealthReportTime" return="long"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <em>last timestamp</em> at which the health report was received.
+ @return <em>last timestamp</em> at which the health report was received]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p><code>NodeHealthStatus</code> is a summary of the health status of the
+ node.</p>
+
+ <p>It includes information such as:
+   <ul>
+     <li>
+       An indicator of whether the node is healthy, as determined by the
+       health-check script.
+     </li>
+     <li>The previous time at which the health status was reported.</li>
+     <li>A diagnostic report on the health status.</li>
+   </ul>
+ </p>
+
+ @see NodeReport
+ @see ApplicationClientProtocol#getClusterNodes(org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest)]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.records.NodeHealthStatus -->
+  <!-- start class org.apache.hadoop.yarn.server.api.records.NodeStatus -->
+  <class name="NodeStatus" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NodeStatus"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="newInstance" return="org.apache.hadoop.yarn.server.api.records.NodeStatus"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <param name="responseId" type="int"/>
+      <param name="containerStatuses" type="java.util.List"/>
+      <param name="keepAliveApplications" type="java.util.List"/>
+      <param name="nodeHealthStatus" type="org.apache.hadoop.yarn.server.api.records.NodeHealthStatus"/>
+    </method>
+    <method name="getNodeId" return="org.apache.hadoop.yarn.api.records.NodeId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getResponseId" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getContainersStatuses" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setContainersStatuses"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containersStatuses" type="java.util.List"/>
+    </method>
+    <method name="getKeepAliveApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setKeepAliveApplications"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appIds" type="java.util.List"/>
+    </method>
+    <method name="getNodeHealthStatus" return="org.apache.hadoop.yarn.server.api.records.NodeHealthStatus"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setNodeHealthStatus"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="healthStatus" type="org.apache.hadoop.yarn.server.api.records.NodeHealthStatus"/>
+    </method>
+    <method name="setNodeId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+    </method>
+    <method name="setResponseId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="responseId" type="int"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.records.NodeStatus -->
+</package>
+<package name="org.apache.hadoop.yarn.server.api.records.impl.pb">
+  <!-- start class org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl -->
+  <class name="MasterKeyPBImpl" extends="org.apache.hadoop.yarn.api.records.impl.pb.ProtoBase"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.server.api.records.MasterKey"/>
+    <constructor name="MasterKeyPBImpl"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="MasterKeyPBImpl" type="org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getKeyId" return="int"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setKeyId"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="id" type="int"/>
+    </method>
+    <method name="getBytes" return="java.nio.ByteBuffer"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setBytes"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="bytes" type="java.nio.ByteBuffer"/>
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="obj" type="java.lang.Object"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl -->
+  <!-- start class org.apache.hadoop.yarn.server.api.records.impl.pb.NodeHealthStatusPBImpl -->
+  <class name="NodeHealthStatusPBImpl" extends="org.apache.hadoop.yarn.server.api.records.NodeHealthStatus"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NodeHealthStatusPBImpl"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="NodeHealthStatusPBImpl" type="org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeHealthStatusProto"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeHealthStatusProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="other" type="java.lang.Object"/>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getIsNodeHealthy" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setIsNodeHealthy"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="isNodeHealthy" type="boolean"/>
+    </method>
+    <method name="getHealthReport" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setHealthReport"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="healthReport" type="java.lang.String"/>
+    </method>
+    <method name="getLastHealthReportTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setLastHealthReportTime"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="lastHealthReport" type="long"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.records.impl.pb.NodeHealthStatusPBImpl -->
+  <!-- start class org.apache.hadoop.yarn.server.api.records.impl.pb.NodeStatusPBImpl -->
+  <class name="NodeStatusPBImpl" extends="org.apache.hadoop.yarn.server.api.records.NodeStatus"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NodeStatusPBImpl"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="NodeStatusPBImpl" type="org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="other" type="java.lang.Object"/>
+    </method>
+    <method name="getResponseId" return="int"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setResponseId"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="responseId" type="int"/>
+    </method>
+    <method name="getNodeId" return="org.apache.hadoop.yarn.api.records.NodeId"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setNodeId"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+    </method>
+    <method name="getContainersStatuses" return="java.util.List"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setContainersStatuses"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containers" type="java.util.List"/>
+    </method>
+    <method name="getKeepAliveApplications" return="java.util.List"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setKeepAliveApplications"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appIds" type="java.util.List"/>
+    </method>
+    <method name="getNodeHealthStatus" return="org.apache.hadoop.yarn.server.api.records.NodeHealthStatus"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setNodeHealthStatus"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="healthStatus" type="org.apache.hadoop.yarn.server.api.records.NodeHealthStatus"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.records.impl.pb.NodeStatusPBImpl -->
+</package>
+<package name="org.apache.hadoop.yarn.server.metrics">
+</package>
+<package name="org.apache.hadoop.yarn.server.records">
+</package>
+<package name="org.apache.hadoop.yarn.server.records.impl.pb">
+  <!-- start class org.apache.hadoop.yarn.server.records.impl.pb.VersionPBImpl -->
+  <class name="VersionPBImpl" extends="org.apache.hadoop.yarn.server.records.Version"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="VersionPBImpl"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="VersionPBImpl" type="org.apache.hadoop.yarn.proto.YarnServerCommonProtos.VersionProto"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnServerCommonProtos.VersionProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getMajorVersion" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setMajorVersion"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="major" type="int"/>
+    </method>
+    <method name="getMinorVersion" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setMinorVersion"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="minor" type="int"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.records.impl.pb.VersionPBImpl -->
+</package>
+<package name="org.apache.hadoop.yarn.server.security.http">
+  <!-- start class org.apache.hadoop.yarn.server.security.http.RMAuthenticationFilterInitializer -->
+  <class name="RMAuthenticationFilterInitializer" extends="org.apache.hadoop.http.FilterInitializer"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="RMAuthenticationFilterInitializer"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createFilterConfig" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="initFilter"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="container" type="org.apache.hadoop.http.FilterContainer"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.security.http.RMAuthenticationFilterInitializer -->
+</package>
+<package name="org.apache.hadoop.yarn.server.utils">
+  <!-- start class org.apache.hadoop.yarn.server.utils.BuilderUtils -->
+  <class name="BuilderUtils" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="BuilderUtils"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="newLocalResource" return="org.apache.hadoop.yarn.api.records.LocalResource"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="url" type="org.apache.hadoop.yarn.api.records.URL"/>
+      <param name="type" type="org.apache.hadoop.yarn.api.records.LocalResourceType"/>
+      <param name="visibility" type="org.apache.hadoop.yarn.api.records.LocalResourceVisibility"/>
+      <param name="size" type="long"/>
+      <param name="timestamp" type="long"/>
+    </method>
+    <method name="newLocalResource" return="org.apache.hadoop.yarn.api.records.LocalResource"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="uri" type="java.net.URI"/>
+      <param name="type" type="org.apache.hadoop.yarn.api.records.LocalResourceType"/>
+      <param name="visibility" type="org.apache.hadoop.yarn.api.records.LocalResourceVisibility"/>
+      <param name="size" type="long"/>
+      <param name="timestamp" type="long"/>
+    </method>
+    <method name="newApplicationId" return="org.apache.hadoop.yarn.api.records.ApplicationId"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="recordFactory" type="org.apache.hadoop.yarn.factories.RecordFactory"/>
+      <param name="clustertimestamp" type="long"/>
+      <param name="id" type="java.lang.CharSequence"/>
+    </method>
+    <method name="newApplicationId" return="org.apache.hadoop.yarn.api.records.ApplicationId"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="recordFactory" type="org.apache.hadoop.yarn.factories.RecordFactory"/>
+      <param name="clusterTimeStamp" type="long"/>
+      <param name="id" type="int"/>
+    </method>
+    <method name="newApplicationId" return="org.apache.hadoop.yarn.api.records.ApplicationId"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="clusterTimeStamp" type="long"/>
+      <param name="id" type="int"/>
+    </method>
+    <method name="newApplicationAttemptId" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="attemptId" type="int"/>
+    </method>
+    <method name="convert" return="org.apache.hadoop.yarn.api.records.ApplicationId"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="clustertimestamp" type="long"/>
+      <param name="id" type="java.lang.CharSequence"/>
+    </method>
+    <method name="newContainerId" return="org.apache.hadoop.yarn.api.records.ContainerId"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <param name="containerId" type="long"/>
+    </method>
+    <method name="newContainerId" return="org.apache.hadoop.yarn.api.records.ContainerId"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="int"/>
+      <param name="appAttemptId" type="int"/>
+      <param name="timestamp" type="long"/>
+      <param name="id" type="long"/>
+    </method>
+    <method name="newContainerToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="cId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <param name="host" type="java.lang.String"/>
+      <param name="port" type="int"/>
+      <param name="user" type="java.lang.String"/>
+      <param name="r" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <param name="expiryTime" type="long"/>
+      <param name="masterKeyId" type="int"/>
+      <param name="password" type="byte[]"/>
+      <param name="rmIdentifier" type="long"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="newContainerId" return="org.apache.hadoop.yarn.api.records.ContainerId"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="recordFactory" type="org.apache.hadoop.yarn.factories.RecordFactory"/>
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="appAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <param name="containerId" type="int"/>
+    </method>
+    <method name="newNodeId" return="org.apache.hadoop.yarn.api.records.NodeId"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="host" type="java.lang.String"/>
+      <param name="port" type="int"/>
+    </method>
+    <method name="newNodeReport" return="org.apache.hadoop.yarn.api.records.NodeReport"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <param name="nodeState" type="org.apache.hadoop.yarn.api.records.NodeState"/>
+      <param name="httpAddress" type="java.lang.String"/>
+      <param name="rackName" type="java.lang.String"/>
+      <param name="used" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <param name="capability" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <param name="numContainers" type="int"/>
+      <param name="healthReport" type="java.lang.String"/>
+      <param name="lastHealthReportTime" type="long"/>
+    </method>
+    <method name="newNodeReport" return="org.apache.hadoop.yarn.api.records.NodeReport"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <param name="nodeState" type="org.apache.hadoop.yarn.api.records.NodeState"/>
+      <param name="httpAddress" type="java.lang.String"/>
+      <param name="rackName" type="java.lang.String"/>
+      <param name="used" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <param name="capability" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <param name="numContainers" type="int"/>
+      <param name="healthReport" type="java.lang.String"/>
+      <param name="lastHealthReportTime" type="long"/>
+      <param name="nodeLabels" type="java.util.Set"/>
+    </method>
+    <method name="newContainerStatus" return="org.apache.hadoop.yarn.api.records.ContainerStatus"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <param name="containerState" type="org.apache.hadoop.yarn.api.records.ContainerState"/>
+      <param name="diagnostics" type="java.lang.String"/>
+      <param name="exitStatus" type="int"/>
+    </method>
+    <method name="newContainer" return="org.apache.hadoop.yarn.api.records.Container"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <param name="nodeHttpAddress" type="java.lang.String"/>
+      <param name="resource" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <param name="priority" type="org.apache.hadoop.yarn.api.records.Priority"/>
+      <param name="containerToken" type="org.apache.hadoop.yarn.api.records.Token"/>
+    </method>
+    <method name="newToken" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="tokenClass" type="java.lang.Class"/>
+      <param name="identifier" type="byte[]"/>
+      <param name="kind" type="java.lang.String"/>
+      <param name="password" type="byte[]"/>
+      <param name="service" type="java.lang.String"/>
+    </method>
+    <method name="newDelegationToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="identifier" type="byte[]"/>
+      <param name="kind" type="java.lang.String"/>
+      <param name="password" type="byte[]"/>
+      <param name="service" type="java.lang.String"/>
+    </method>
+    <method name="newClientToAMToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="identifier" type="byte[]"/>
+      <param name="kind" type="java.lang.String"/>
+      <param name="password" type="byte[]"/>
+      <param name="service" type="java.lang.String"/>
+    </method>
+    <method name="newAMRMToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="identifier" type="byte[]"/>
+      <param name="kind" type="java.lang.String"/>
+      <param name="password" type="byte[]"/>
+      <param name="service" type="java.lang.String"/>
+    </method>
+    <method name="newContainerTokenIdentifier" return="org.apache.hadoop.yarn.security.ContainerTokenIdentifier"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerToken" type="org.apache.hadoop.yarn.api.records.Token"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="newContainerLaunchContext" return="org.apache.hadoop.yarn.api.records.ContainerLaunchContext"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="localResources" type="java.util.Map"/>
+      <param name="environment" type="java.util.Map"/>
+      <param name="commands" type="java.util.List"/>
+      <param name="serviceData" type="java.util.Map"/>
+      <param name="tokens" type="java.nio.ByteBuffer"/>
+      <param name="acls" type="java.util.Map"/>
+    </method>
+    <method name="newPriority" return="org.apache.hadoop.yarn.api.records.Priority"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="p" type="int"/>
+    </method>
+    <method name="newResourceRequest" return="org.apache.hadoop.yarn.api.records.ResourceRequest"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="priority" type="org.apache.hadoop.yarn.api.records.Priority"/>
+      <param name="hostName" type="java.lang.String"/>
+      <param name="capability" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <param name="numContainers" type="int"/>
+    </method>
+    <method name="newResourceRequest" return="org.apache.hadoop.yarn.api.records.ResourceRequest"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="r" type="org.apache.hadoop.yarn.api.records.ResourceRequest"/>
+    </method>
+    <method name="newApplicationReport" return="org.apache.hadoop.yarn.api.records.ApplicationReport"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <param name="user" type="java.lang.String"/>
+      <param name="queue" type="java.lang.String"/>
+      <param name="name" type="java.lang.String"/>
+      <param name="host" type="java.lang.String"/>
+      <param name="rpcPort" type="int"/>
+      <param name="clientToAMToken" type="org.apache.hadoop.yarn.api.records.Token"/>
+      <param name="state" type="org.apache.hadoop.yarn.api.records.YarnApplicationState"/>
+      <param name="diagnostics" type="java.lang.String"/>
+      <param name="url" type="java.lang.String"/>
+      <param name="startTime" type="long"/>
+      <param name="finishTime" type="long"/>
+      <param name="finalStatus" type="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"/>
+      <param name="appResources" type="org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport"/>
+      <param name="origTrackingUrl" type="java.lang.String"/>
+      <param name="progress" type="float"/>
+      <param name="appType" type="java.lang.String"/>
+      <param name="amRmToken" type="org.apache.hadoop.yarn.api.records.Token"/>
+      <param name="tags" type="java.util.Set"/>
+    </method>
+    <method name="newApplicationSubmissionContext" return="org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="applicationName" type="java.lang.String"/>
+      <param name="queue" type="java.lang.String"/>
+      <param name="priority" type="org.apache.hadoop.yarn.api.records.Priority"/>
+      <param name="amContainer" type="org.apache.hadoop.yarn.api.records.ContainerLaunchContext"/>
+      <param name="isUnmanagedAM" type="boolean"/>
+      <param name="cancelTokensWhenComplete" type="boolean"/>
+      <param name="maxAppAttempts" type="int"/>
+      <param name="resource" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <param name="applicationType" type="java.lang.String"/>
+    </method>
+    <method name="newApplicationSubmissionContext" return="org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="applicationName" type="java.lang.String"/>
+      <param name="queue" type="java.lang.String"/>
+      <param name="priority" type="org.apache.hadoop.yarn.api.records.Priority"/>
+      <param name="amContainer" type="org.apache.hadoop.yarn.api.records.ContainerLaunchContext"/>
+      <param name="isUnmanagedAM" type="boolean"/>
+      <param name="cancelTokensWhenComplete" type="boolean"/>
+      <param name="maxAppAttempts" type="int"/>
+      <param name="resource" type="org.apache.hadoop.yarn.api.records.Resource"/>
+    </method>
+    <method name="newApplicationResourceUsageReport" return="org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="numUsedContainers" type="int"/>
+      <param name="numReservedContainers" type="int"/>
+      <param name="usedResources" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <param name="reservedResources" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <param name="neededResources" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <param name="memorySeconds" type="long"/>
+      <param name="vcoreSeconds" type="long"/>
+    </method>
+    <method name="newResource" return="org.apache.hadoop.yarn.api.records.Resource"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="memory" type="int"/>
+      <param name="vCores" type="int"/>
+    </method>
+    <method name="newURL" return="org.apache.hadoop.yarn.api.records.URL"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="scheme" type="java.lang.String"/>
+      <param name="host" type="java.lang.String"/>
+      <param name="port" type="int"/>
+      <param name="file" type="java.lang.String"/>
+    </method>
+    <method name="newAllocateResponse" return="org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="responseId" type="int"/>
+      <param name="completedContainers" type="java.util.List"/>
+      <param name="allocatedContainers" type="java.util.List"/>
+      <param name="updatedNodes" type="java.util.List"/>
+      <param name="availResources" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <param name="command" type="org.apache.hadoop.yarn.api.records.AMCommand"/>
+      <param name="numClusterNodes" type="int"/>
+      <param name="preempt" type="org.apache.hadoop.yarn.api.records.PreemptionMessage"/>
+    </method>
+    <doc>
+    <![CDATA[Builder utilities to construct various objects.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.utils.BuilderUtils -->
+  <!-- start class org.apache.hadoop.yarn.server.utils.BuilderUtils.ApplicationIdComparator -->
+  <class name="BuilderUtils.ApplicationIdComparator" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.util.Comparator"/>
+    <implements name="java.io.Serializable"/>
+    <constructor name="BuilderUtils.ApplicationIdComparator"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="compare" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="a1" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="a2" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.utils.BuilderUtils.ApplicationIdComparator -->
+  <!-- start class org.apache.hadoop.yarn.server.utils.BuilderUtils.ContainerIdComparator -->
+  <class name="BuilderUtils.ContainerIdComparator" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.util.Comparator"/>
+    <implements name="java.io.Serializable"/>
+    <constructor name="BuilderUtils.ContainerIdComparator"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="compare" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="c1" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <param name="c2" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.utils.BuilderUtils.ContainerIdComparator -->
+  <!-- start class org.apache.hadoop.yarn.server.utils.LeveldbIterator -->
+  <class name="LeveldbIterator" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.util.Iterator"/>
+    <implements name="java.io.Closeable"/>
+    <constructor name="LeveldbIterator" type="org.iq80.leveldb.DB"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create an iterator for the specified database]]>
+      </doc>
+    </constructor>
+    <constructor name="LeveldbIterator" type="org.iq80.leveldb.DB, org.iq80.leveldb.ReadOptions"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create an iterator for the specified database]]>
+      </doc>
+    </constructor>
+    <constructor name="LeveldbIterator" type="org.iq80.leveldb.DBIterator"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create an iterator using the specified underlying DBIterator]]>
+      </doc>
+    </constructor>
+    <method name="seek"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="key" type="byte[]"/>
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Repositions the iterator so the key of the next BlockElement
+ returned greater than or equal to the specified targetKey.]]>
+      </doc>
+    </method>
+    <method name="seekToFirst"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Repositions the iterator so is is at the beginning of the Database.]]>
+      </doc>
+    </method>
+    <method name="seekToLast"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Repositions the iterator so it is at the end of of the Database.]]>
+      </doc>
+    </method>
+    <method name="hasNext" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Returns <tt>true</tt> if the iteration has more elements.]]>
+      </doc>
+    </method>
+    <method name="next" return="java.util.Map.Entry"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Returns the next element in the iteration.]]>
+      </doc>
+    </method>
+    <method name="peekNext" return="java.util.Map.Entry"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Returns the next element in the iteration, without advancing the
+ iteration.]]>
+      </doc>
+    </method>
+    <method name="hasPrev" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[@return true if there is a previous entry in the iteration.]]>
+      </doc>
+    </method>
+    <method name="prev" return="java.util.Map.Entry"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[@return the previous element in the iteration and rewinds the iteration.]]>
+      </doc>
+    </method>
+    <method name="peekPrev" return="java.util.Map.Entry"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[@return the previous element in the iteration, without rewinding the
+ iteration.]]>
+      </doc>
+    </method>
+    <method name="remove"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Removes from the database the last element returned by the iterator.]]>
+      </doc>
+    </method>
+    <method name="close"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Closes the iterator.]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[A wrapper for a DBIterator to translate the raw RuntimeExceptions that
+ can be thrown into DBExceptions.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.utils.LeveldbIterator -->
+  <!-- start class org.apache.hadoop.yarn.server.utils.Lock -->
+  <class name="Lock"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.lang.annotation.Annotation"/>
+    <doc>
+    <![CDATA[Annotation to document locking order.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.utils.Lock -->
+  <!-- start class org.apache.hadoop.yarn.server.utils.Lock.NoLock -->
+  <class name="Lock.NoLock" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="Lock.NoLock"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.utils.Lock.NoLock -->
+  <!-- start class org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils -->
+  <class name="YarnServerBuilderUtils" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="YarnServerBuilderUtils"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="newNodeHeartbeatResponse" return="org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="responseId" type="int"/>
+      <param name="action" type="org.apache.hadoop.yarn.server.api.records.NodeAction"/>
+      <param name="containersToCleanUp" type="java.util.List"/>
+      <param name="applicationsToCleanUp" type="java.util.List"/>
+      <param name="containerTokenMasterKey" type="org.apache.hadoop.yarn.server.api.records.MasterKey"/>
+      <param name="nmTokenMasterKey" type="org.apache.hadoop.yarn.server.api.records.MasterKey"/>
+      <param name="nextHeartbeatInterval" type="long"/>
+    </method>
+    <doc>
+    <![CDATA[Server Builder utilities to construct various objects.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils -->
+</package>
+<package name="org.apache.hadoop.yarn.server.webapp">
+  <!-- start class org.apache.hadoop.yarn.server.webapp.AppAttemptBlock -->
+  <class name="AppAttemptBlock" extends="org.apache.hadoop.yarn.webapp.view.HtmlBlock"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AppAttemptBlock" type="org.apache.hadoop.yarn.server.api.ApplicationContext"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="render"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="html" type="org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.AppAttemptBlock -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.AppBlock -->
+  <class name="AppBlock" extends="org.apache.hadoop.yarn.webapp.view.HtmlBlock"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="render"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="html" type="org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block"/>
+    </method>
+    <field name="appContext" type="org.apache.hadoop.yarn.server.api.ApplicationContext"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.AppBlock -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.AppsBlock -->
+  <class name="AppsBlock" extends="org.apache.hadoop.yarn.webapp.view.HtmlBlock"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="render"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="html" type="org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block"/>
+    </method>
+    <field name="appContext" type="org.apache.hadoop.yarn.server.api.ApplicationContext"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.AppsBlock -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.ContainerBlock -->
+  <class name="ContainerBlock" extends="org.apache.hadoop.yarn.webapp.view.HtmlBlock"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ContainerBlock" type="org.apache.hadoop.yarn.server.api.ApplicationContext, org.apache.hadoop.yarn.webapp.View.ViewContext"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="render"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="html" type="org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.ContainerBlock -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.WebServices -->
+  <class name="WebServices" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="WebServices" type="org.apache.hadoop.yarn.server.api.ApplicationContext"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getApps" return="org.apache.hadoop.yarn.server.webapp.dao.AppsInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="javax.servlet.http.HttpServletRequest"/>
+      <param name="res" type="javax.servlet.http.HttpServletResponse"/>
+      <param name="stateQuery" type="java.lang.String"/>
+      <param name="statesQuery" type="java.util.Set"/>
+      <param name="finalStatusQuery" type="java.lang.String"/>
+      <param name="userQuery" type="java.lang.String"/>
+      <param name="queueQuery" type="java.lang.String"/>
+      <param name="count" type="java.lang.String"/>
+      <param name="startedBegin" type="java.lang.String"/>
+      <param name="startedEnd" type="java.lang.String"/>
+      <param name="finishBegin" type="java.lang.String"/>
+      <param name="finishEnd" type="java.lang.String"/>
+      <param name="applicationTypes" type="java.util.Set"/>
+    </method>
+    <method name="getApp" return="org.apache.hadoop.yarn.server.webapp.dao.AppInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="javax.servlet.http.HttpServletRequest"/>
+      <param name="res" type="javax.servlet.http.HttpServletResponse"/>
+      <param name="appId" type="java.lang.String"/>
+    </method>
+    <method name="getAppAttempts" return="org.apache.hadoop.yarn.server.webapp.dao.AppAttemptsInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="javax.servlet.http.HttpServletRequest"/>
+      <param name="res" type="javax.servlet.http.HttpServletResponse"/>
+      <param name="appId" type="java.lang.String"/>
+    </method>
+    <method name="getAppAttempt" return="org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="javax.servlet.http.HttpServletRequest"/>
+      <param name="res" type="javax.servlet.http.HttpServletResponse"/>
+      <param name="appId" type="java.lang.String"/>
+      <param name="appAttemptId" type="java.lang.String"/>
+    </method>
+    <method name="getContainers" return="org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="javax.servlet.http.HttpServletRequest"/>
+      <param name="res" type="javax.servlet.http.HttpServletResponse"/>
+      <param name="appId" type="java.lang.String"/>
+      <param name="appAttemptId" type="java.lang.String"/>
+    </method>
+    <method name="getContainer" return="org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="javax.servlet.http.HttpServletRequest"/>
+      <param name="res" type="javax.servlet.http.HttpServletResponse"/>
+      <param name="appId" type="java.lang.String"/>
+      <param name="appAttemptId" type="java.lang.String"/>
+      <param name="containerId" type="java.lang.String"/>
+    </method>
+    <method name="init"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="response" type="javax.servlet.http.HttpServletResponse"/>
+    </method>
+    <method name="parseQueries" return="java.util.Set"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="queries" type="java.util.Set"/>
+      <param name="isState" type="boolean"/>
+    </method>
+    <method name="parseApplicationId" return="org.apache.hadoop.yarn.api.records.ApplicationId"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="appId" type="java.lang.String"/>
+    </method>
+    <method name="parseApplicationAttemptId" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="appAttemptId" type="java.lang.String"/>
+    </method>
+    <method name="parseContainerId" return="org.apache.hadoop.yarn.api.records.ContainerId"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="containerId" type="java.lang.String"/>
+    </method>
+    <method name="validateIds"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="appAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+    </method>
+    <method name="getUser" return="org.apache.hadoop.security.UserGroupInformation"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="req" type="javax.servlet.http.HttpServletRequest"/>
+    </method>
+    <field name="appContext" type="org.apache.hadoop.yarn.server.api.ApplicationContext"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.WebServices -->
+</package>
+<package name="org.apache.hadoop.yarn.server.webapp.dao">
+  <!-- start class org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo -->
+  <class name="AppAttemptInfo" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AppAttemptInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AppAttemptInfo" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptReport"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getAppAttemptId" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getHost" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getRpcPort" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getTrackingUrl" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getOriginalTrackingUrl" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getDiagnosticsInfo" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAppAttemptState" return="org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAmContainerId" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="appAttemptId" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="host" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="rpcPort" type="int"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="trackingUrl" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="originalTrackingUrl" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="diagnosticsInfo" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="appAttemptState" type="org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="amContainerId" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protect

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[50/51] [abbrv] hadoop git commit: YARN-4829. Add support for binary units in Resource class.(vvasudev via asuresh)

Posted by vv...@apache.org.
YARN-4829. Add support for binary units in Resource class.(vvasudev via asuresh)


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

Branch: refs/heads/YARN-3926
Commit: dd046eb6c8a2ba7c515e57fc3367549d2fe6108f
Parents: 0bc39d3
Author: Arun Suresh <as...@apache.org>
Authored: Thu Mar 17 23:50:22 2016 -0700
Committer: Varun Vasudev <vv...@apache.org>
Committed: Mon Jun 13 14:42:02 2016 +0530

----------------------------------------------------------------------
 .../yarn/api/records/ResourceInformation.java   |  2 +-
 .../hadoop/yarn/util/UnitsConversionUtil.java   | 45 ++++++++++++++------
 .../yarn/util/TestUnitsConversionUtil.java      | 17 +++++++-
 .../api/records/impl/pb/ResourcePBImpl.java     |  2 +-
 .../yarn/util/resource/TestResourceUtils.java   |  2 +-
 5 files changed, 52 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd046eb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
index 80e3192..a17e81b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
@@ -36,7 +36,7 @@ public class ResourceInformation implements Comparable<ResourceInformation> {
   private static final String VCORES_URI = "vcores";
 
   public static final ResourceInformation MEMORY_MB =
-      ResourceInformation.newInstance(MEMORY_URI, "M");
+      ResourceInformation.newInstance(MEMORY_URI, "Mi");
   public static final ResourceInformation VCORES =
       ResourceInformation.newInstance(VCORES_URI);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd046eb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/UnitsConversionUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/UnitsConversionUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/UnitsConversionUtil.java
index 7785263..47bb3df 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/UnitsConversionUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/UnitsConversionUtil.java
@@ -46,7 +46,8 @@ public class UnitsConversionUtil {
   }
 
   private static final String[] UNITS =
-      {"p", "n", "u", "m", "", "k", "M", "G", "T", "P"};
+      { "p", "n", "u", "m", "", "k", "M", "G", "T", "P", "Ki", "Mi", "Gi", "Ti",
+          "Pi" };
   private static final List<String> SORTED_UNITS = Arrays.asList(UNITS);
   public static final Set<String> KNOWN_UNITS = createKnownUnitsSet();
   private static final Converter PICO =
@@ -65,6 +66,15 @@ public class UnitsConversionUtil {
   private static final Converter PETA =
       new Converter(1000L * 1000L * 1000L * 1000L * 1000L, 1L);
 
+  private static final Converter KILO_BINARY = new Converter(1024L, 1L);
+  private static final Converter MEGA_BINARY = new Converter(1024L * 1024L, 1L);
+  private static final Converter GIGA_BINARY =
+      new Converter(1024L * 1024L * 1024L, 1L);
+  private static final Converter TERA_BINARY =
+      new Converter(1024L * 1024L * 1024L * 1024L, 1L);
+  private static final Converter PETA_BINARY =
+      new Converter(1024L * 1024L * 1024L * 1024L * 1024L, 1L);
+
   private static Set<String> createKnownUnitsSet() {
     Set<String> ret = new HashSet<>();
     ret.addAll(Arrays.asList(UNITS));
@@ -93,6 +103,16 @@ public class UnitsConversionUtil {
       return TERA;
     case "P":
       return PETA;
+    case "Ki":
+      return KILO_BINARY;
+    case "Mi":
+      return MEGA_BINARY;
+    case "Gi":
+      return GIGA_BINARY;
+    case "Ti":
+      return TERA_BINARY;
+    case "Pi":
+      return PETA_BINARY;
     default:
       throw new IllegalArgumentException(
           "Unknown unit '" + unit + "'. Known units are " + KNOWN_UNITS);
@@ -112,28 +132,29 @@ public class UnitsConversionUtil {
     if (toUnit == null || fromUnit == null || fromValue == null) {
       throw new IllegalArgumentException("One or more arguments are null");
     }
-    Long tmp;
     String overflowMsg =
         "Converting " + fromValue + " from '" + fromUnit + "' to '" + toUnit
             + "' will result in an overflow of Long";
+    if (fromUnit.equals(toUnit)) {
+      return fromValue;
+    }
     Converter fc = getConverter(fromUnit);
     Converter tc = getConverter(toUnit);
     Long numerator = fc.numerator * tc.denominator;
     Long denominator = fc.denominator * tc.numerator;
+    Long numeratorMultiplierLimit = Long.MAX_VALUE / numerator;
     if (numerator < denominator) {
-      if (!toUnit.equals(fromUnit)) {
-        tmp = Long.MAX_VALUE / numerator;
-        if (tmp < fromValue) {
-          throw new IllegalArgumentException(overflowMsg);
-        }
+      if (numeratorMultiplierLimit < fromValue) {
+        throw new IllegalArgumentException(overflowMsg);
       }
       return (fromValue * numerator) / denominator;
     }
-    tmp = numerator / denominator;
-    if (!toUnit.equals(fromUnit)) {
-      if ((Long.MAX_VALUE / tmp) < fromValue) {
-        throw new IllegalArgumentException(overflowMsg);
-      }
+    if (numeratorMultiplierLimit > fromValue) {
+      return (numerator * fromValue) / denominator;
+    }
+    Long tmp = numerator / denominator;
+    if ((Long.MAX_VALUE / tmp) < fromValue) {
+      throw new IllegalArgumentException(overflowMsg);
     }
     return fromValue * tmp;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd046eb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/util/TestUnitsConversionUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/util/TestUnitsConversionUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/util/TestUnitsConversionUtil.java
index 421768f..0f999e8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/util/TestUnitsConversionUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/util/TestUnitsConversionUtil.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.yarn.util;
 
-import org.apache.hadoop.yarn.util.UnitsConversionUtil;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -45,6 +44,7 @@ public class TestUnitsConversionUtil {
     fromUnit = "";
     Assert.assertEquals("kilo test failed", Long.valueOf(test / 1000l),
         UnitsConversionUtil.convert(fromUnit, "k", test));
+
     Assert
         .assertEquals("mega test failed", Long.valueOf(test / (1000l * 1000l)),
             UnitsConversionUtil.convert(fromUnit, "M", test));
@@ -63,6 +63,21 @@ public class TestUnitsConversionUtil {
 
     Assert.assertEquals("mega to giga test failed", Long.valueOf(value),
         UnitsConversionUtil.convert("M", "G", Long.valueOf(value * 1000l)));
+
+    Assert.assertEquals("Mi to Gi test failed", Long.valueOf(value),
+        UnitsConversionUtil.convert("Mi", "Gi", Long.valueOf(value * 1024l)));
+
+    Assert.assertEquals("Mi to Ki test failed", Long.valueOf(value * 1024),
+        UnitsConversionUtil.convert("Mi", "Ki", Long.valueOf(value)));
+
+    Assert.assertEquals("Ki to base units test failed", Long.valueOf(5 * 1024),
+        UnitsConversionUtil.convert("Ki", "", Long.valueOf(5)));
+
+    Assert.assertEquals("Mi to k test failed", Long.valueOf(1073741),
+        UnitsConversionUtil.convert("Mi", "k", Long.valueOf(1024)));
+
+    Assert.assertEquals("M to Mi test failed", Long.valueOf(953),
+        UnitsConversionUtil.convert("M", "Mi", Long.valueOf(1000)));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd046eb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
index 34e3607..f4ca882 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
@@ -81,7 +81,7 @@ public class ResourcePBImpl extends Resource {
     initResourcesMap();
     ResourceInformation ri =
         this.getResourceInformation(ResourceInformation.MEMORY_MB.getName());
-    return UnitsConversionUtil.convert(ri.getUnits(), "M", ri.getValue());
+    return UnitsConversionUtil.convert(ri.getUnits(), "Mi", ri.getValue());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd046eb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java
index 606e89c..4bb68c0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java
@@ -195,7 +195,7 @@ public class TestResourceUtils {
       // we must always have memory and vcores with their fixed units
       Assert.assertTrue(ret.containsKey("memory-mb"));
       ResourceInformation memInfo = ret.get("memory-mb");
-      Assert.assertEquals("M", memInfo.getUnits());
+      Assert.assertEquals("Mi", memInfo.getUnits());
       Assert.assertEquals(ResourceTypes.COUNTABLE, memInfo.getResourceType());
       Assert.assertTrue(ret.containsKey("vcores"));
       ResourceInformation vcoresInfo = ret.get("vcores");


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[40/51] [abbrv] hadoop git commit: YARN-5124. Modify AMRMClient to set the ExecutionType in the ResourceRequest. (asuresh)

Posted by vv...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/51432779/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestDistributedScheduling.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestDistributedScheduling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestDistributedScheduling.java
index 6d93eb3..a556aa2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestDistributedScheduling.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestDistributedScheduling.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
  * 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.
@@ -22,20 +22,31 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.service.Service;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NMToken;
+import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
@@ -43,12 +54,23 @@ import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.junit.After;
 import org.junit.Assert;
-import org.junit.Ignore;
+import org.junit.Before;
 import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 /**
  * Validates End2End Distributed Scheduling flow which includes the AM
@@ -57,11 +79,70 @@ import java.util.List;
  * the NM and the DistributedSchedulingProtocol used by the framework to talk
  * to the DistributedSchedulingService running on the RM.
  */
-public class TestDistributedScheduling extends TestAMRMProxy {
+public class TestDistributedScheduling extends BaseAMRMProxyE2ETest {
 
   private static final Log LOG =
       LogFactory.getLog(TestDistributedScheduling.class);
 
+  protected MiniYARNCluster cluster;
+  protected YarnClient rmClient;
+  protected ApplicationMasterProtocol client;
+  protected Configuration conf;
+  protected Configuration yarnConf;
+  protected ApplicationAttemptId attemptId;
+  protected ApplicationId appId;
+
+  @Before
+  public void doBefore() throws Exception {
+    cluster = new MiniYARNCluster("testDistributedSchedulingE2E", 1, 1, 1);
+
+    conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.AMRM_PROXY_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.DIST_SCHEDULING_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.NM_CONTAINER_QUEUING_ENABLED, true);
+    cluster.init(conf);
+    cluster.start();
+    yarnConf = cluster.getConfig();
+
+    // the client has to connect to AMRMProxy
+    yarnConf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS,
+        YarnConfiguration.DEFAULT_AMRM_PROXY_ADDRESS);
+    rmClient = YarnClient.createYarnClient();
+    rmClient.init(yarnConf);
+    rmClient.start();
+
+    // Submit application
+    attemptId = createApp(rmClient, cluster, conf);
+    appId = attemptId.getApplicationId();
+    client = createAMRMProtocol(rmClient, appId, cluster, yarnConf);
+  }
+
+  @After
+  public void doAfter() throws Exception {
+    if (client != null) {
+      try {
+        client.finishApplicationMaster(FinishApplicationMasterRequest
+            .newInstance(FinalApplicationStatus.SUCCEEDED, "success", null));
+        rmClient.killApplication(attemptId.getApplicationId());
+        attemptId = null;
+      } catch (Exception e) {
+      }
+    }
+    if (rmClient != null) {
+      try {
+        rmClient.stop();
+      } catch (Exception e) {
+      }
+    }
+    if (cluster != null) {
+      try {
+        cluster.stop();
+      } catch (Exception e) {
+      }
+    }
+  }
+
+
   /**
    * Validates if Allocate Requests containing only OPPORTUNISTIC container
    * requests are satisfied instantly.
@@ -70,104 +151,63 @@ public class TestDistributedScheduling extends TestAMRMProxy {
    */
   @Test(timeout = 60000)
   public void testOpportunisticExecutionTypeRequestE2E() throws Exception {
-    MiniYARNCluster cluster =
-        new MiniYARNCluster("testDistributedSchedulingE2E", 1, 1, 1);
-    YarnClient rmClient = null;
-    ApplicationMasterProtocol client;
-
-    try {
-      Configuration conf = new YarnConfiguration();
-      conf.setBoolean(YarnConfiguration.AMRM_PROXY_ENABLED, true);
-      conf.setBoolean(YarnConfiguration.DIST_SCHEDULING_ENABLED, true);
-      conf.setBoolean(YarnConfiguration.NM_CONTAINER_QUEUING_ENABLED, true);
-      cluster.init(conf);
-      cluster.start();
-      final Configuration yarnConf = cluster.getConfig();
-
-      // the client has to connect to AMRMProxy
-
-      yarnConf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS,
-          YarnConfiguration.DEFAULT_AMRM_PROXY_ADDRESS);
-      rmClient = YarnClient.createYarnClient();
-      rmClient.init(yarnConf);
-      rmClient.start();
-
-      // Submit application
-
-      ApplicationId appId = createApp(rmClient, cluster);
-
-      client = createAMRMProtocol(rmClient, appId, cluster, yarnConf);
-
-      LOG.info("testDistributedSchedulingE2E - Register");
-
-      RegisterApplicationMasterResponse responseRegister =
-          client.registerApplicationMaster(RegisterApplicationMasterRequest
-              .newInstance(NetUtils.getHostname(), 1024, ""));
-
-      Assert.assertNotNull(responseRegister);
-      Assert.assertNotNull(responseRegister.getQueue());
-      Assert.assertNotNull(responseRegister.getApplicationACLs());
-      Assert.assertNotNull(responseRegister.getClientToAMTokenMasterKey());
-      Assert
-          .assertNotNull(responseRegister.getContainersFromPreviousAttempts());
-      Assert.assertNotNull(responseRegister.getSchedulerResourceTypes());
-      Assert.assertNotNull(responseRegister.getMaximumResourceCapability());
-
-      RMApp rmApp =
-          cluster.getResourceManager().getRMContext().getRMApps().get(appId);
-      Assert.assertEquals(RMAppState.RUNNING, rmApp.getState());
-
-      LOG.info("testDistributedSchedulingE2E - Allocate");
-
-      AllocateRequest request =
-          createAllocateRequest(rmClient.getNodeReports(NodeState.RUNNING));
-
-      // Replace 'ANY' requests with OPPORTUNISTIC aks and remove
-      // everything else
-      List<ResourceRequest> newAskList = new ArrayList<>();
-      for (ResourceRequest rr : request.getAskList()) {
-        if (ResourceRequest.ANY.equals(rr.getResourceName())) {
-          ResourceRequest newRR = ResourceRequest.newInstance(rr
-                  .getPriority(), rr.getResourceName(),
-              rr.getCapability(), rr.getNumContainers(), rr.getRelaxLocality(),
-              rr.getNodeLabelExpression(),
-              ExecutionTypeRequest.newInstance(
-                  ExecutionType.OPPORTUNISTIC, true));
-          newAskList.add(newRR);
-        }
-      }
-      request.setAskList(newAskList);
-
-      AllocateResponse allocResponse = client.allocate(request);
-      Assert.assertNotNull(allocResponse);
-
-      // Ensure that all the requests are satisfied immediately
-      Assert.assertEquals(2, allocResponse.getAllocatedContainers().size());
-
-      // Verify that the allocated containers are OPPORTUNISTIC
-      for (Container allocatedContainer : allocResponse
-          .getAllocatedContainers()) {
-        ContainerTokenIdentifier containerTokenIdentifier = BuilderUtils
-            .newContainerTokenIdentifier(
-                allocatedContainer.getContainerToken());
-        Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
-            containerTokenIdentifier.getExecutionType());
-      }
-
-      LOG.info("testDistributedSchedulingE2E - Finish");
-
-      FinishApplicationMasterResponse responseFinish =
-          client.finishApplicationMaster(FinishApplicationMasterRequest
-              .newInstance(FinalApplicationStatus.SUCCEEDED, "success", null));
-
-      Assert.assertNotNull(responseFinish);
-
-    } finally {
-      if (rmClient != null) {
-        rmClient.stop();
+    LOG.info("testDistributedSchedulingE2E - Register");
+
+    RegisterApplicationMasterResponse responseRegister =
+        client.registerApplicationMaster(RegisterApplicationMasterRequest
+            .newInstance(NetUtils.getHostname(), 1024, ""));
+
+    Assert.assertNotNull(responseRegister);
+    Assert.assertNotNull(responseRegister.getQueue());
+    Assert.assertNotNull(responseRegister.getApplicationACLs());
+    Assert.assertNotNull(responseRegister.getClientToAMTokenMasterKey());
+    Assert
+        .assertNotNull(responseRegister.getContainersFromPreviousAttempts());
+    Assert.assertNotNull(responseRegister.getSchedulerResourceTypes());
+    Assert.assertNotNull(responseRegister.getMaximumResourceCapability());
+
+    RMApp rmApp =
+        cluster.getResourceManager().getRMContext().getRMApps().get(appId);
+    Assert.assertEquals(RMAppState.RUNNING, rmApp.getState());
+
+    LOG.info("testDistributedSchedulingE2E - Allocate");
+
+    AllocateRequest request =
+        createAllocateRequest(rmClient.getNodeReports(NodeState.RUNNING));
+
+    // Replace 'ANY' requests with OPPORTUNISTIC aks and remove
+    // everything else
+    List<ResourceRequest> newAskList = new ArrayList<>();
+    for (ResourceRequest rr : request.getAskList()) {
+      if (ResourceRequest.ANY.equals(rr.getResourceName())) {
+        ResourceRequest newRR = ResourceRequest.newInstance(rr
+                .getPriority(), rr.getResourceName(),
+            rr.getCapability(), rr.getNumContainers(), rr.getRelaxLocality(),
+            rr.getNodeLabelExpression(),
+            ExecutionTypeRequest.newInstance(
+                ExecutionType.OPPORTUNISTIC, true));
+        newAskList.add(newRR);
       }
-      cluster.stop();
     }
+    request.setAskList(newAskList);
+
+    AllocateResponse allocResponse = client.allocate(request);
+    Assert.assertNotNull(allocResponse);
+
+    // Ensure that all the requests are satisfied immediately
+    Assert.assertEquals(2, allocResponse.getAllocatedContainers().size());
+
+    // Verify that the allocated containers are OPPORTUNISTIC
+    for (Container allocatedContainer : allocResponse
+        .getAllocatedContainers()) {
+      ContainerTokenIdentifier containerTokenIdentifier = BuilderUtils
+          .newContainerTokenIdentifier(
+              allocatedContainer.getContainerToken());
+      Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
+          containerTokenIdentifier.getExecutionType());
+    }
+
+    LOG.info("testDistributedSchedulingE2E - Finish");
   }
 
   /**
@@ -178,135 +218,305 @@ public class TestDistributedScheduling extends TestAMRMProxy {
    */
   @Test(timeout = 60000)
   public void testMixedExecutionTypeRequestE2E() throws Exception {
-    MiniYARNCluster cluster =
-        new MiniYARNCluster("testDistributedSchedulingE2E", 1, 1, 1);
-    YarnClient rmClient = null;
-    ApplicationMasterProtocol client;
+    LOG.info("testDistributedSchedulingE2E - Register");
+
+    RegisterApplicationMasterResponse responseRegister =
+        client.registerApplicationMaster(RegisterApplicationMasterRequest
+            .newInstance(NetUtils.getHostname(), 1024, ""));
+
+    Assert.assertNotNull(responseRegister);
+    Assert.assertNotNull(responseRegister.getQueue());
+    Assert.assertNotNull(responseRegister.getApplicationACLs());
+    Assert.assertNotNull(responseRegister.getClientToAMTokenMasterKey());
+    Assert
+        .assertNotNull(responseRegister.getContainersFromPreviousAttempts());
+    Assert.assertNotNull(responseRegister.getSchedulerResourceTypes());
+    Assert.assertNotNull(responseRegister.getMaximumResourceCapability());
+
+    RMApp rmApp =
+        cluster.getResourceManager().getRMContext().getRMApps().get(appId);
+    Assert.assertEquals(RMAppState.RUNNING, rmApp.getState());
+
+    LOG.info("testDistributedSchedulingE2E - Allocate");
+
+    AllocateRequest request =
+        createAllocateRequest(rmClient.getNodeReports(NodeState.RUNNING));
+    List<ResourceRequest> askList = request.getAskList();
+    List<ResourceRequest> newAskList = new ArrayList<>(askList);
+
+    // Duplicate all ANY requests marking them as opportunistic
+    for (ResourceRequest rr : askList) {
+      if (ResourceRequest.ANY.equals(rr.getResourceName())) {
+        ResourceRequest newRR = ResourceRequest.newInstance(rr
+                .getPriority(), rr.getResourceName(),
+            rr.getCapability(), rr.getNumContainers(), rr.getRelaxLocality(),
+            rr.getNodeLabelExpression(),
+            ExecutionTypeRequest.newInstance(
+                ExecutionType.OPPORTUNISTIC, true));
+        newAskList.add(newRR);
+      }
+    }
+    request.setAskList(newAskList);
+
+    AllocateResponse allocResponse = client.allocate(request);
+    Assert.assertNotNull(allocResponse);
+
+    // Ensure that all the requests are satisfied immediately
+    Assert.assertEquals(2, allocResponse.getAllocatedContainers().size());
+
+    // Verify that the allocated containers are OPPORTUNISTIC
+    for (Container allocatedContainer : allocResponse
+        .getAllocatedContainers()) {
+      ContainerTokenIdentifier containerTokenIdentifier = BuilderUtils
+          .newContainerTokenIdentifier(
+              allocatedContainer.getContainerToken());
+      Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
+          containerTokenIdentifier.getExecutionType());
+    }
+
+    request.setAskList(new ArrayList<ResourceRequest>());
+    request.setResponseId(request.getResponseId() + 1);
 
+    Thread.sleep(1000);
+
+    // RM should allocate GUARANTEED containers within 2 calls to allocate()
+    allocResponse = client.allocate(request);
+    Assert.assertNotNull(allocResponse);
+    Assert.assertEquals(2, allocResponse.getAllocatedContainers().size());
+
+    // Verify that the allocated containers are GUARANTEED
+    for (Container allocatedContainer : allocResponse
+        .getAllocatedContainers()) {
+      ContainerTokenIdentifier containerTokenIdentifier = BuilderUtils
+          .newContainerTokenIdentifier(
+              allocatedContainer.getContainerToken());
+      Assert.assertEquals(ExecutionType.GUARANTEED,
+          containerTokenIdentifier.getExecutionType());
+    }
+
+    LOG.info("testDistributedSchedulingE2E - Finish");
+  }
+
+  /**
+   * Validates if AMRMClient can be used with Distributed Scheduling turned on.
+   *
+   * @throws Exception
+   */
+  @Test(timeout = 120000)
+  @SuppressWarnings("unchecked")
+  public void testAMRMClient() throws Exception {
+    AMRMClientImpl<AMRMClient.ContainerRequest> amClient = null;
     try {
-      Configuration conf = new YarnConfiguration();
-      conf.setBoolean(YarnConfiguration.AMRM_PROXY_ENABLED, true);
-      conf.setBoolean(YarnConfiguration.DIST_SCHEDULING_ENABLED, true);
-      conf.setBoolean(YarnConfiguration.NM_CONTAINER_QUEUING_ENABLED, true);
-      cluster.init(conf);
-      cluster.start();
-      final Configuration yarnConf = cluster.getConfig();
-
-      // the client has to connect to AMRMProxy
-
-      yarnConf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS,
-          YarnConfiguration.DEFAULT_AMRM_PROXY_ADDRESS);
-      rmClient = YarnClient.createYarnClient();
-      rmClient.init(yarnConf);
-      rmClient.start();
-
-      // Submit application
-
-      ApplicationId appId = createApp(rmClient, cluster);
-
-      client = createAMRMProtocol(rmClient, appId, cluster, yarnConf);
-
-      LOG.info("testDistributedSchedulingE2E - Register");
-
-      RegisterApplicationMasterResponse responseRegister =
-          client.registerApplicationMaster(RegisterApplicationMasterRequest
-              .newInstance(NetUtils.getHostname(), 1024, ""));
-
-      Assert.assertNotNull(responseRegister);
-      Assert.assertNotNull(responseRegister.getQueue());
-      Assert.assertNotNull(responseRegister.getApplicationACLs());
-      Assert.assertNotNull(responseRegister.getClientToAMTokenMasterKey());
-      Assert
-          .assertNotNull(responseRegister.getContainersFromPreviousAttempts());
-      Assert.assertNotNull(responseRegister.getSchedulerResourceTypes());
-      Assert.assertNotNull(responseRegister.getMaximumResourceCapability());
-
-      RMApp rmApp =
-          cluster.getResourceManager().getRMContext().getRMApps().get(appId);
-      Assert.assertEquals(RMAppState.RUNNING, rmApp.getState());
-
-      LOG.info("testDistributedSchedulingE2E - Allocate");
-
-      AllocateRequest request =
-          createAllocateRequest(rmClient.getNodeReports(NodeState.RUNNING));
-      List<ResourceRequest> askList = request.getAskList();
-      List<ResourceRequest> newAskList = new ArrayList<>(askList);
-
-      // Duplicate all ANY requests marking them as opportunistic
-      for (ResourceRequest rr : askList) {
-        if (ResourceRequest.ANY.equals(rr.getResourceName())) {
-          ResourceRequest newRR = ResourceRequest.newInstance(rr
-              .getPriority(), rr.getResourceName(),
-              rr.getCapability(), rr.getNumContainers(), rr.getRelaxLocality(),
-              rr.getNodeLabelExpression(),
+      Priority priority = Priority.newInstance(1);
+      Priority priority2 = Priority.newInstance(2);
+      Resource capability = Resource.newInstance(1024, 1);
+
+      List<NodeReport> nodeReports = rmClient.getNodeReports(NodeState.RUNNING);
+      String node = nodeReports.get(0).getNodeId().getHost();
+      String rack = nodeReports.get(0).getRackName();
+      String[] nodes = new String[]{node};
+      String[] racks = new String[]{rack};
+
+      // start am rm client
+      amClient = new AMRMClientImpl(client);
+      amClient.init(yarnConf);
+      amClient.start();
+      amClient.registerApplicationMaster(NetUtils.getHostname(), 1024, "");
+
+      assertEquals(0, amClient.ask.size());
+      assertEquals(0, amClient.release.size());
+
+      amClient.addContainerRequest(
+          new AMRMClient.ContainerRequest(capability, nodes, racks, priority));
+      amClient.addContainerRequest(
+          new AMRMClient.ContainerRequest(capability, nodes, racks, priority));
+      amClient.addContainerRequest(
+          new AMRMClient.ContainerRequest(capability, nodes, racks, priority));
+      amClient.addContainerRequest(
+          new AMRMClient.ContainerRequest(capability, nodes, racks, priority));
+      amClient.addContainerRequest(
+          new AMRMClient.ContainerRequest(capability, null, null, priority2,
+              true, null,
+              ExecutionTypeRequest.newInstance(
+                  ExecutionType.OPPORTUNISTIC, true)));
+      amClient.addContainerRequest(
+          new AMRMClient.ContainerRequest(capability, null, null, priority2,
+              true, null,
+              ExecutionTypeRequest.newInstance(
+                  ExecutionType.OPPORTUNISTIC, true)));
+
+      amClient.removeContainerRequest(
+          new AMRMClient.ContainerRequest(capability, nodes, racks, priority));
+      amClient.removeContainerRequest(
+          new AMRMClient.ContainerRequest(capability, nodes, racks, priority));
+      amClient.removeContainerRequest(
+          new AMRMClient.ContainerRequest(capability, null, null, priority2,
+              true, null,
               ExecutionTypeRequest.newInstance(
-                  ExecutionType.OPPORTUNISTIC, true));
-          newAskList.add(newRR);
+                  ExecutionType.OPPORTUNISTIC, true)));
+
+      int containersRequestedNode = amClient.remoteRequestsTable.get(priority,
+          node, ExecutionType.GUARANTEED, capability).remoteRequest
+          .getNumContainers();
+      int containersRequestedRack = amClient.remoteRequestsTable.get(priority,
+          rack, ExecutionType.GUARANTEED, capability).remoteRequest
+          .getNumContainers();
+      int containersRequestedAny = amClient.remoteRequestsTable.get(priority,
+          ResourceRequest.ANY, ExecutionType.GUARANTEED, capability)
+          .remoteRequest.getNumContainers();
+      int oppContainersRequestedAny =
+          amClient.remoteRequestsTable.get(priority2, ResourceRequest.ANY,
+              ExecutionType.OPPORTUNISTIC, capability).remoteRequest
+              .getNumContainers();
+
+      assertEquals(2, containersRequestedNode);
+      assertEquals(2, containersRequestedRack);
+      assertEquals(2, containersRequestedAny);
+      assertEquals(1, oppContainersRequestedAny);
+
+      assertEquals(4, amClient.ask.size());
+      assertEquals(0, amClient.release.size());
+
+      // RM should allocate container within 2 calls to allocate()
+      int allocatedContainerCount = 0;
+      int iterationsLeft = 10;
+      Set<ContainerId> releases = new TreeSet<>();
+
+      amClient.getNMTokenCache().clearCache();
+      Assert.assertEquals(0,
+          amClient.getNMTokenCache().numberOfTokensInCache());
+      HashMap<String, Token> receivedNMTokens = new HashMap<>();
+
+      while (allocatedContainerCount <
+          (containersRequestedAny + oppContainersRequestedAny)
+          && iterationsLeft-- > 0) {
+        AllocateResponse allocResponse = amClient.allocate(0.1f);
+        assertEquals(0, amClient.ask.size());
+        assertEquals(0, amClient.release.size());
+
+        allocatedContainerCount += allocResponse.getAllocatedContainers()
+            .size();
+        for (Container container : allocResponse.getAllocatedContainers()) {
+          ContainerId rejectContainerId = container.getId();
+          releases.add(rejectContainerId);
         }
-      }
-      request.setAskList(newAskList);
-
-      AllocateResponse allocResponse = client.allocate(request);
-      Assert.assertNotNull(allocResponse);
-
-      // Ensure that all the requests are satisfied immediately
-      Assert.assertEquals(2, allocResponse.getAllocatedContainers().size());
-
-      // Verify that the allocated containers are OPPORTUNISTIC
-      for (Container allocatedContainer : allocResponse
-          .getAllocatedContainers()) {
-        ContainerTokenIdentifier containerTokenIdentifier = BuilderUtils
-            .newContainerTokenIdentifier(
-            allocatedContainer.getContainerToken());
-        Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
-            containerTokenIdentifier.getExecutionType());
-      }
-
-      request.setAskList(new ArrayList<ResourceRequest>());
-      request.setResponseId(request.getResponseId() + 1);
-
-      Thread.sleep(1000);
 
-      // RM should allocate GUARANTEED containers within 2 calls to allocate()
-      allocResponse = client.allocate(request);
-      Assert.assertNotNull(allocResponse);
-      Assert.assertEquals(2, allocResponse.getAllocatedContainers().size());
+        for (NMToken token : allocResponse.getNMTokens()) {
+          String nodeID = token.getNodeId().toString();
+          receivedNMTokens.put(nodeID, token.getToken());
+        }
 
-      // Verify that the allocated containers are GUARANTEED
-      for (Container allocatedContainer : allocResponse
-          .getAllocatedContainers()) {
-        ContainerTokenIdentifier containerTokenIdentifier = BuilderUtils
-            .newContainerTokenIdentifier(
-                allocatedContainer.getContainerToken());
-        Assert.assertEquals(ExecutionType.GUARANTEED,
-            containerTokenIdentifier.getExecutionType());
+        if (allocatedContainerCount < containersRequestedAny) {
+          // sleep to let NM's heartbeat to RM and trigger allocations
+          sleep(100);
+        }
       }
 
-      LOG.info("testDistributedSchedulingE2E - Finish");
+      assertEquals(allocatedContainerCount,
+          containersRequestedAny + oppContainersRequestedAny);
+      for (ContainerId rejectContainerId : releases) {
+        amClient.releaseAssignedContainer(rejectContainerId);
+      }
+      assertEquals(3, amClient.release.size());
+      assertEquals(0, amClient.ask.size());
+
+      // need to tell the AMRMClient that we dont need these resources anymore
+      amClient.removeContainerRequest(
+          new AMRMClient.ContainerRequest(capability, nodes, racks, priority));
+      amClient.removeContainerRequest(
+          new AMRMClient.ContainerRequest(capability, nodes, racks, priority));
+      amClient.removeContainerRequest(
+          new AMRMClient.ContainerRequest(capability, nodes, racks, priority2,
+              true, null,
+              ExecutionTypeRequest.newInstance(
+                  ExecutionType.OPPORTUNISTIC, true)));
+      assertEquals(4, amClient.ask.size());
+
+      // test RPC exception handling
+      amClient.addContainerRequest(new AMRMClient.ContainerRequest(capability,
+          nodes, racks, priority));
+      amClient.addContainerRequest(new AMRMClient.ContainerRequest(capability,
+          nodes, racks, priority));
+      amClient.addContainerRequest(
+          new AMRMClient.ContainerRequest(capability, nodes, racks, priority2,
+              true, null,
+              ExecutionTypeRequest.newInstance(
+                  ExecutionType.OPPORTUNISTIC, true)));
+
+      final AMRMClient amc = amClient;
+      ApplicationMasterProtocol realRM = amClient.rmClient;
+      try {
+        ApplicationMasterProtocol mockRM = mock(ApplicationMasterProtocol
+            .class);
+        when(mockRM.allocate(any(AllocateRequest.class))).thenAnswer(
+            new Answer<AllocateResponse>() {
+              public AllocateResponse answer(InvocationOnMock invocation)
+                  throws Exception {
+                amc.removeContainerRequest(
+                    new AMRMClient.ContainerRequest(capability, nodes,
+                        racks, priority));
+                amc.removeContainerRequest(
+                    new AMRMClient.ContainerRequest(capability, nodes, racks,
+                        priority));
+                amc.removeContainerRequest(
+                    new AMRMClient.ContainerRequest(capability, null, null,
+                        priority2, true, null,
+                        ExecutionTypeRequest.newInstance(
+                            ExecutionType.OPPORTUNISTIC, true)));
+                throw new Exception();
+              }
+            });
+        amClient.rmClient = mockRM;
+        amClient.allocate(0.1f);
+      } catch (Exception ioe) {
+      } finally {
+        amClient.rmClient = realRM;
+      }
 
-      FinishApplicationMasterResponse responseFinish =
-          client.finishApplicationMaster(FinishApplicationMasterRequest
-              .newInstance(FinalApplicationStatus.SUCCEEDED, "success", null));
+      assertEquals(3, amClient.release.size());
+      assertEquals(6, amClient.ask.size());
+
+      iterationsLeft = 3;
+      // do a few iterations to ensure RM is not going send new containers
+      while (iterationsLeft-- > 0) {
+        // inform RM of rejection
+        AllocateResponse allocResponse = amClient.allocate(0.1f);
+        // RM did not send new containers because AM does not need any
+        assertEquals(0, allocResponse.getAllocatedContainers().size());
+        if (allocResponse.getCompletedContainersStatuses().size() > 0) {
+          for (ContainerStatus cStatus : allocResponse
+              .getCompletedContainersStatuses()) {
+            if (releases.contains(cStatus.getContainerId())) {
+              assertEquals(cStatus.getState(), ContainerState.COMPLETE);
+              assertEquals(-100, cStatus.getExitStatus());
+              releases.remove(cStatus.getContainerId());
+            }
+          }
+        }
+        if (iterationsLeft > 0) {
+          // sleep to make sure NM's heartbeat
+          sleep(100);
+        }
+      }
+      assertEquals(0, amClient.ask.size());
+      assertEquals(0, amClient.release.size());
 
-      Assert.assertNotNull(responseFinish);
+      amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
+          null, null);
 
     } finally {
-      if (rmClient != null) {
-        rmClient.stop();
+      if (amClient != null && amClient.getServiceState() == Service.STATE
+          .STARTED) {
+        amClient.stop();
       }
-      cluster.stop();
     }
   }
 
-  @Ignore
-  @Override
-  public void testAMRMProxyE2E() throws Exception { }
-
-  @Ignore
-  @Override
-  public void testE2ETokenRenewal() throws Exception { }
-
-  @Ignore
-  @Override
-  public void testE2ETokenSwap() throws Exception { }
+  private void sleep(int sleepTime) {
+    try {
+      Thread.sleep(sleepTime);
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51432779/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
index cd04130..969fb70 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
@@ -251,9 +252,9 @@ public class TestNMClient {
           racks, priority));
     }
 
-    int containersRequestedAny = rmClient.remoteRequestsTable.get(priority)
-        .get(ResourceRequest.ANY).get(capability).remoteRequest
-        .getNumContainers();
+    int containersRequestedAny = rmClient.remoteRequestsTable.get(priority,
+        ResourceRequest.ANY, ExecutionType.GUARANTEED, capability)
+        .remoteRequest.getNumContainers();
 
     // RM should allocate container within 2 calls to allocate()
     int allocatedContainerCount = 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51432779/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
index fd56f4f..b0c4b97 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
@@ -214,7 +214,8 @@ public class ResourceRequestPBImpl extends  ResourceRequest {
         + ", # Containers: " + getNumContainers()
         + ", Location: " + getResourceName()
         + ", Relax Locality: " + getRelaxLocality()
-        + ", Execution Spec: " + getExecutionTypeRequest() + "}";
+        + ", Execution Type Request: " + getExecutionTypeRequest()
+        + ", Node Label Expression: " + getNodeLabelExpression() + "}";
   }
 
   @Override
@@ -235,4 +236,4 @@ public class ResourceRequestPBImpl extends  ResourceRequest {
     }
     builder.setNodeLabelExpression(nodeLabelExpression);
   }
-}
\ No newline at end of file
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[47/51] [abbrv] hadoop git commit: YARN-4081. Add support for multiple resource types in the Resource class. (Varun Vasudev via wangda)

Posted by vv...@apache.org.
YARN-4081. Add support for multiple resource types in the Resource class. (Varun Vasudev via wangda)


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

Branch: refs/heads/YARN-3926
Commit: 9e518ef1ea6227e5425fe69a8285b01148e678b8
Parents: 28b66ae
Author: Wangda Tan <wa...@apache.org>
Authored: Thu Sep 10 09:43:26 2015 -0700
Committer: Varun Vasudev <vv...@apache.org>
Committed: Mon Jun 13 14:25:08 2016 +0530

----------------------------------------------------------------------
 .../src/main/resources/META-INF/LICENSE.txt     | 1661 ++++++++++++++++++
 .../src/main/resources/META-INF/NOTICE.txt      |  283 +++
 .../yarn/api/protocolrecords/ResourceTypes.java |   27 +
 .../hadoop/yarn/api/records/Resource.java       |  205 ++-
 .../yarn/api/records/ResourceInformation.java   |  218 +++
 .../exceptions/ResourceNotFoundException.java   |   45 +
 .../hadoop/yarn/util/UnitsConversionUtil.java   |  197 +++
 .../src/main/proto/yarn_protos.proto            |   12 +
 .../yarn/conf/TestResourceInformation.java      |   70 +
 .../yarn/util/TestUnitsConversionUtil.java      |  120 ++
 .../yarn/api/records/impl/pb/ProtoUtils.java    |   13 +
 .../api/records/impl/pb/ResourcePBImpl.java     |  197 ++-
 .../hadoop/yarn/util/resource/Resources.java    |  129 +-
 .../hadoop/yarn/api/TestPBImplRecords.java      |    3 +
 14 files changed, 3095 insertions(+), 85 deletions(-)
----------------------------------------------------------------------



---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[43/51] [abbrv] hadoop git commit: HADOOP-13243. TestRollingFileSystemSink.testSetInitialFlushTime() fails intermittently. (Daniel Templeton via kasha)

Posted by vv...@apache.org.
HADOOP-13243. TestRollingFileSystemSink.testSetInitialFlushTime() fails intermittently. (Daniel Templeton via kasha)


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

Branch: refs/heads/YARN-3926
Commit: 49b4064644b242921af4ddf1de9932bc7bcf5f0e
Parents: 7dae2b3
Author: Karthik Kambatla <ka...@cloudera.com>
Authored: Sun Jun 12 14:12:31 2016 -0700
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Sun Jun 12 14:12:31 2016 -0700

----------------------------------------------------------------------
 .../hadoop/metrics2/sink/TestRollingFileSystemSink.java      | 8 +++-----
 1 file changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/49b40646/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestRollingFileSystemSink.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestRollingFileSystemSink.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestRollingFileSystemSink.java
index 9c34dba..1273052 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestRollingFileSystemSink.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestRollingFileSystemSink.java
@@ -118,15 +118,14 @@ public class TestRollingFileSystemSink {
     diff = rfsSink.nextFlush.getTimeInMillis() - calendar.getTimeInMillis();
 
     assertTrue("The initial flush time was calculated incorrectly: " + diff,
-        (diff >= -1000L) && (diff < -900L));
+        (diff == 0L) || ((diff > -1000L) && (diff < -900L)));
 
     calendar.set(Calendar.MILLISECOND, 10);
     rfsSink.setInitialFlushTime(calendar.getTime());
     diff = rfsSink.nextFlush.getTimeInMillis() - calendar.getTimeInMillis();
 
     assertTrue("The initial flush time was calculated incorrectly: " + diff,
-        ((diff >= -10L) && (diff <= 0L) ||
-            ((diff > -1000L) && (diff < -910L))));
+        (diff >= -10L) && (diff <= 0L) || ((diff > -1000L) && (diff < -910L)));
 
     calendar.set(Calendar.SECOND, 1);
     calendar.set(Calendar.MILLISECOND, 10);
@@ -134,8 +133,7 @@ public class TestRollingFileSystemSink {
     diff = rfsSink.nextFlush.getTimeInMillis() - calendar.getTimeInMillis();
 
     assertTrue("The initial flush time was calculated incorrectly: " + diff,
-        ((diff >= -10L) && (diff <= 0L) ||
-            ((diff > -1000L) && (diff < -910L))));
+        (diff >= -10L) && (diff <= 0L) || ((diff > -1000L) && (diff < -910L)));
 
     // Now try pathological settings
     rfsSink = new RollingFileSystemSink(1000, 1000000);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[33/51] [abbrv] hadoop git commit: YARN-3426. Add jdiff support to YARN. (vinodkv via wangda)

Posted by vv...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/03fc6b1b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_API_2.7.2.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_API_2.7.2.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_API_2.7.2.xml
new file mode 100644
index 0000000..ff01b26
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_API_2.7.2.xml
@@ -0,0 +1,13692 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Thu May 12 17:47:18 PDT 2016 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="hadoop-yarn-api 2.7.2"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.ExcludePrivateAnnotationsJDiffDoclet -docletpath /Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/hadoop-annotations.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/jdiff.jar -verbose -classpath /Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/classes:/Users/vinodkv/.m2/repository/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/Users/vinodkv/.m2/repository/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/Users/vinodkv/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/vinodkv/.m2/repository/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-common-project/hadoop-common/target/hadoop-common-2.7.2
 .jar:/Users/vinodkv/.m2/repository/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/Users/vinodkv/.m2/repository/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/Users/vinodkv/.m2/repository/xmlenc/xmlenc/0.52/xmlenc-0.52.jar:/Users/vinodkv/.m2/repository/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1.jar:/Users/vinodkv/.m2/repository/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/Users/vinodkv/.m2/repository/commons-io/commons-io/2.4/commons-io-2.4.jar:/Users/vinodkv/.m2/repository/commons-net/commons-net/3.1/commons-net-3.1.jar:/Users/vinodkv/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/Users/vinodkv/.m2/repository/javax/servlet/servlet-api/2.5/servlet-api-2.5.jar:/Users/vinodkv/.m2/repository/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/Users/vinodkv/.m2/repository/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/Users/vinodkv/.m2/repository/javax/servlet/jsp/jsp-api/2.1/jsp-api-
 2.1.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/jersey-core/1.9/jersey-core-1.9.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/jersey-json/1.9/jersey-json-1.9.jar:/Users/vinodkv/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/Users/vinodkv/.m2/repository/com/sun/xml/bind/jaxb-impl/2.2.3-1/jaxb-impl-2.2.3-1.jar:/Users/vinodkv/.m2/repository/javax/xml/bind/jaxb-api/2.2.2/jaxb-api-2.2.2.jar:/Users/vinodkv/.m2/repository/javax/xml/stream/stax-api/1.0-2/stax-api-1.0-2.jar:/Users/vinodkv/.m2/repository/javax/activation/activation/1.1/activation-1.1.jar:/Users/vinodkv/.m2/repository/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/Users/vinodkv/.m2/repository/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/jersey-server/1.9/jersey-server-1.9.jar:/Users/vinodkv/.m2/repository/asm/asm/3.2/asm-3.2.jar:/Users/vinodkv/.m2/repository/log4j/log4j/1.2.17/log4j-1.2.17.jar:/Users/vinodkv/.m2/repos
 itory/net/java/dev/jets3t/jets3t/0.9.0/jets3t-0.9.0.jar:/Users/vinodkv/.m2/repository/org/apache/httpcomponents/httpclient/4.2.5/httpclient-4.2.5.jar:/Users/vinodkv/.m2/repository/org/apache/httpcomponents/httpcore/4.2.5/httpcore-4.2.5.jar:/Users/vinodkv/.m2/repository/com/jamesmurty/utils/java-xmlbuilder/0.4/java-xmlbuilder-0.4.jar:/Users/vinodkv/.m2/repository/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/Users/vinodkv/.m2/repository/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/Users/vinodkv/.m2/repository/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/Users/vinodkv/.m2/repository/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar:/Users/vinodkv/.m2/repository/org/slf4j/slf4j-api/1.7.10/slf4j-api-1.7.10.jar:/Users/vinodkv/.m2/repository/org/slf4j/slf4j-log4j12/1.7.10/slf4j-log4j12-1.7.10.jar:/Users/vinodkv/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core
 -asl-1.9.13.jar:/Users/vinodkv/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/vinodkv/.m2/repository/org/apache/avro/avro/1.7.4/avro-1.7.4.jar:/Users/vinodkv/.m2/repository/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/Users/vinodkv/.m2/repository/org/xerial/snappy/snappy-java/1.0.4.1/snappy-java-1.0.4.1.jar:/Users/vinodkv/.m2/repository/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-common-project/hadoop-auth/target/hadoop-auth-2.7.2.jar:/Users/vinodkv/.m2/repository/org/apache/directory/server/apacheds-kerberos-codec/2.0.0-M15/apacheds-kerberos-codec-2.0.0-M15.jar:/Users/vinodkv/.m2/repository/org/apache/directory/server/apacheds-i18n/2.0.0-M15/apacheds-i18n-2.0.0-M15.jar:/Users/vinodkv/.m2/repository/org/apache/directory/api/api-asn1-api/1.0.0-M20/api-asn1-api-1.0.0-M20.jar:/Users/vinodkv/.m2/repository/org/apache/directory/api/api-util/1.0.0-M20/a
 pi-util-1.0.0-M20.jar:/Users/vinodkv/.m2/repository/org/apache/curator/curator-framework/2.7.1/curator-framework-2.7.1.jar:/Users/vinodkv/.m2/repository/com/jcraft/jsch/0.1.42/jsch-0.1.42.jar:/Users/vinodkv/.m2/repository/org/apache/curator/curator-client/2.7.1/curator-client-2.7.1.jar:/Users/vinodkv/.m2/repository/org/apache/curator/curator-recipes/2.7.1/curator-recipes-2.7.1.jar:/Users/vinodkv/.m2/repository/org/apache/htrace/htrace-core/3.1.0-incubating/htrace-core-3.1.0-incubating.jar:/Users/vinodkv/.m2/repository/org/apache/zookeeper/zookeeper/3.4.6/zookeeper-3.4.6.jar:/Users/vinodkv/.m2/repository/io/netty/netty/3.6.2.Final/netty-3.6.2.Final.jar:/Users/vinodkv/.m2/repository/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/Users/vinodkv/.m2/repository/org/tukaani/xz/1.0/xz-1.0.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-common-project/hadoop-annotations/target/hadoop-annotations-2.7.2.jar:/Library/Java/JavaVirtualMachines/jdk1.7
 .0_45.jdk/Contents/Home/lib/tools.jar:/Users/vinodkv/.m2/repository/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar -sourcepath /Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java -apidir /Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/site/jdiff/xml -apiname hadoop-yarn-api 2.7.2 -->
+<package name="org.apache.hadoop.yarn.api">
+  <!-- start interface org.apache.hadoop.yarn.api.ApplicationClientProtocol -->
+  <interface name="ApplicationClientProtocol"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.api.ApplicationBaseProtocol"/>
+    <method name="getNewApplication" return="org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by clients to obtain a new {@link ApplicationId} for
+ submitting new applications.</p>
+
+ <p>The <code>ResourceManager</code> responds with a new, monotonically
+ increasing, {@link ApplicationId} which is used by the client to submit
+ a new application.</p>
+
+ <p>The <code>ResourceManager</code> also responds with details such
+ as maximum resource capabilities in the cluster as specified in
+ {@link GetNewApplicationResponse}.</p>
+
+ @param request request to get a new <code>ApplicationId</code>
+ @return response containing the new <code>ApplicationId</code> to be used
+ to submit an application
+ @throws YarnException
+ @throws IOException
+ @see #submitApplication(SubmitApplicationRequest)]]>
+      </doc>
+    </method>
+    <method name="submitApplication" return="org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by clients to submit a new application to the
+ <code>ResourceManager.</code></p>
+
+ <p>The client is required to provide details such as queue,
+ {@link Resource} required to run the <code>ApplicationMaster</code>,
+ the equivalent of {@link ContainerLaunchContext} for launching
+ the <code>ApplicationMaster</code> etc. via the
+ {@link SubmitApplicationRequest}.</p>
+
+ <p>Currently the <code>ResourceManager</code> sends an immediate (empty)
+ {@link SubmitApplicationResponse} on accepting the submission and throws
+ an exception if it rejects the submission. However, this call needs to be
+ followed by {@link #getApplicationReport(GetApplicationReportRequest)}
+ to make sure that the application gets properly submitted - obtaining a
+ {@link SubmitApplicationResponse} from ResourceManager doesn't guarantee
+ that RM 'remembers' this application beyond failover or restart. If RM
+ failover or RM restart happens before ResourceManager saves the
+ application's state successfully, the subsequent
+ {@link #getApplicationReport(GetApplicationReportRequest)} will throw
+ a {@link ApplicationNotFoundException}. The Clients need to re-submit
+ the application with the same {@link ApplicationSubmissionContext} when
+ it encounters the {@link ApplicationNotFoundException} on the
+ {@link #getApplicationReport(GetApplicationReportRequest)} call.</p>
+
+ <p>During the submission process, it checks whether the application
+ already exists. If the application exists, it will simply return
+ SubmitApplicationResponse</p>
+
+ <p> In secure mode,the <code>ResourceManager</code> verifies access to
+ queues etc. before accepting the application submission.</p>
+
+ @param request request to submit a new application
+ @return (empty) response on accepting the submission
+ @throws YarnException
+ @throws IOException
+ @see #getNewApplication(GetNewApplicationRequest)]]>
+      </doc>
+    </method>
+    <method name="forceKillApplication" return="org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by clients to request the
+ <code>ResourceManager</code> to abort submitted application.</p>
+
+ <p>The client, via {@link KillApplicationRequest} provides the
+ {@link ApplicationId} of the application to be aborted.</p>
+
+ <p> In secure mode,the <code>ResourceManager</code> verifies access to the
+ application, queue etc. before terminating the application.</p>
+
+ <p>Currently, the <code>ResourceManager</code> returns an empty response
+ on success and throws an exception on rejecting the request.</p>
+
+ @param request request to abort a submitted application
+ @return <code>ResourceManager</code> returns an empty response
+         on success and throws an exception on rejecting the request
+ @throws YarnException
+ @throws IOException
+ @see #getQueueUserAcls(GetQueueUserAclsInfoRequest)]]>
+      </doc>
+    </method>
+    <method name="getClusterMetrics" return="org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by clients to get metrics about the cluster from
+ the <code>ResourceManager</code>.</p>
+
+ <p>The <code>ResourceManager</code> responds with a
+ {@link GetClusterMetricsResponse} which includes the
+ {@link YarnClusterMetrics} with details such as number of current
+ nodes in the cluster.</p>
+
+ @param request request for cluster metrics
+ @return cluster metrics
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getClusterNodes" return="org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by clients to get a report of all nodes
+ in the cluster from the <code>ResourceManager</code>.</p>
+
+ <p>The <code>ResourceManager</code> responds with a
+ {@link GetClusterNodesResponse} which includes the
+ {@link NodeReport} for all the nodes in the cluster.</p>
+
+ @param request request for report on all nodes
+ @return report on all nodes
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getQueueInfo" return="org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by clients to get information about <em>queues</em>
+ from the <code>ResourceManager</code>.</p>
+
+ <p>The client, via {@link GetQueueInfoRequest}, can ask for details such
+ as used/total resources, child queues, running applications etc.</p>
+
+ <p> In secure mode,the <code>ResourceManager</code> verifies access before
+ providing the information.</p>
+
+ @param request request to get queue information
+ @return queue information
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getQueueUserAcls" return="org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by clients to get information about <em>queue
+ acls</em> for <em>current user</em> from the <code>ResourceManager</code>.
+ </p>
+
+ <p>The <code>ResourceManager</code> responds with queue acls for all
+ existing queues.</p>
+
+ @param request request to get queue acls for <em>current user</em>
+ @return queue acls for <em>current user</em>
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="moveApplicationAcrossQueues" return="org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Move an application to a new queue.
+
+ @param request the application ID and the target queue
+ @return an empty response
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="submitReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to submit a new reservation to the
+ {@code ResourceManager}.
+ </p>
+
+ <p>
+ The client packages all details of its request in a
+ {@link ReservationSubmissionRequest} object. This contains information
+ about the amount of capacity, temporal constraints, and concurrency needs.
+ Furthermore, the reservation might be composed of multiple stages, with
+ ordering dependencies among them.
+ </p>
+
+ <p>
+ In order to respond, a new admission control component in the
+ {@code ResourceManager} performs an analysis of the resources that have
+ been committed over the period of time the user is requesting, verify that
+ the user requests can be fulfilled, and that it respect a sharing policy
+ (e.g., {@code CapacityOverTimePolicy}). Once it has positively determined
+ that the ReservationSubmissionRequest is satisfiable the
+ {@code ResourceManager} answers with a
+ {@link ReservationSubmissionResponse} that include a non-null
+ {@link ReservationId}. Upon failure to find a valid allocation the response
+ is an exception with the reason.
+
+ On application submission the client can use this {@link ReservationId} to
+ obtain access to the reserved resources.
+ </p>
+
+ <p>
+ The system guarantees that during the time-range specified by the user, the
+ reservationID will be corresponding to a valid reservation. The amount of
+ capacity dedicated to such queue can vary overtime, depending of the
+ allocation that has been determined. But it is guaranteed to satisfy all
+ the constraint expressed by the user in the
+ {@link ReservationSubmissionRequest}.
+ </p>
+
+ @param request the request to submit a new Reservation
+ @return response the {@link ReservationId} on accepting the submission
+ @throws YarnException if the request is invalid or reservation cannot be
+           created successfully
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="updateReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to update an existing Reservation. This is
+ referred to as a re-negotiation process, in which a user that has
+ previously submitted a Reservation.
+ </p>
+
+ <p>
+ The allocation is attempted by virtually substituting all previous
+ allocations related to this Reservation with new ones, that satisfy the new
+ {@link ReservationUpdateRequest}. Upon success the previous allocation is
+ substituted by the new one, and on failure (i.e., if the system cannot find
+ a valid allocation for the updated request), the previous allocation
+ remains valid.
+
+ The {@link ReservationId} is not changed, and applications currently
+ running within this reservation will automatically receive the resources
+ based on the new allocation.
+ </p>
+
+ @param request to update an existing Reservation (the ReservationRequest
+          should refer to an existing valid {@link ReservationId})
+ @return response empty on successfully updating the existing reservation
+ @throws YarnException if the request is invalid or reservation cannot be
+           updated successfully
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="deleteReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to remove an existing Reservation.
+
+ Upon deletion of a reservation applications running with this reservation,
+ are automatically downgraded to normal jobs running without any dedicated
+ reservation.
+ </p>
+
+ @param request to remove an existing Reservation (the ReservationRequest
+          should refer to an existing valid {@link ReservationId})
+ @return response empty on successfully deleting the existing reservation
+ @throws YarnException if the request is invalid or reservation cannot be
+           deleted successfully
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getNodeToLabels" return="org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by client to get node to labels mappings in existing cluster
+ </p>
+
+ @param request
+ @return node to labels mappings
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getLabelsToNodes" return="org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by client to get labels to nodes mappings
+ in existing cluster
+ </p>
+
+ @param request
+ @return labels to nodes mappings
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getClusterNodeLabels" return="org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by client to get node labels in the cluster
+ </p>
+
+ @param request to get node labels collection of this cluster
+ @return node labels collection of this cluster
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>The protocol between clients and the <code>ResourceManager</code>
+ to submit/abort jobs and to get information on applications, cluster metrics,
+ nodes, queues and ACLs.</p>]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.api.ApplicationClientProtocol -->
+  <!-- start interface org.apache.hadoop.yarn.api.ApplicationConstants -->
+  <interface name="ApplicationConstants"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <field name="APP_SUBMIT_TIME_ENV" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[The environment variable for APP_SUBMIT_TIME. Set in AppMaster environment
+ only]]>
+      </doc>
+    </field>
+    <field name="CONTAINER_TOKEN_FILE_ENV_NAME" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[The cache file into which container token is written]]>
+      </doc>
+    </field>
+    <field name="APPLICATION_WEB_PROXY_BASE_ENV" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[The environmental variable for APPLICATION_WEB_PROXY_BASE. Set in
+ ApplicationMaster's environment only. This states that for all non-relative
+ web URLs in the app masters web UI what base should they have.]]>
+      </doc>
+    </field>
+    <field name="LOG_DIR_EXPANSION_VAR" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[The temporary environmental variable for container log directory. This
+ should be replaced by real container log directory on container launch.]]>
+      </doc>
+    </field>
+    <field name="CLASS_PATH_SEPARATOR" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[This constant is used to construct class path and it will be replaced with
+ real class path separator(':' for Linux and ';' for Windows) by
+ NodeManager on container launch. User has to use this constant to construct
+ class path if user wants cross-platform practice i.e. submit an application
+ from a Windows client to a Linux/Unix server or vice versa.]]>
+      </doc>
+    </field>
+    <field name="PARAMETER_EXPANSION_LEFT" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[The following two constants are used to expand parameter and it will be
+ replaced with real parameter expansion marker ('%' for Windows and '$' for
+ Linux) by NodeManager on container launch. For example: {{VAR}} will be
+ replaced as $VAR on Linux, and %VAR% on Windows. User has to use this
+ constant to construct class path if user wants cross-platform practice i.e.
+ submit an application from a Windows client to a Linux/Unix server or vice
+ versa.]]>
+      </doc>
+    </field>
+    <field name="PARAMETER_EXPANSION_RIGHT" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[User has to use this constant to construct class path if user wants
+ cross-platform practice i.e. submit an application from a Windows client to
+ a Linux/Unix server or vice versa.]]>
+      </doc>
+    </field>
+    <field name="STDERR" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="STDOUT" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="MAX_APP_ATTEMPTS_ENV" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[The environment variable for MAX_APP_ATTEMPTS. Set in AppMaster environment
+ only]]>
+      </doc>
+    </field>
+    <doc>
+    <![CDATA[This is the API for the applications comprising of constants that YARN sets
+ up for the applications and the containers.
+
+ TODO: Investigate the semantics and security of each cross-boundary refs.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.api.ApplicationConstants -->
+  <!-- start class org.apache.hadoop.yarn.api.ApplicationConstants.Environment -->
+  <class name="ApplicationConstants.Environment" extends="java.lang.Enum"
+    abstract="false"
+    static="true" final="true" visibility="public"
+    deprecated="not deprecated">
+    <method name="values" return="org.apache.hadoop.yarn.api.ApplicationConstants.Environment[]"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="valueOf" return="org.apache.hadoop.yarn.api.ApplicationConstants.Environment"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+    </method>
+    <method name="key" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="$" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Expand the environment variable based on client OS environment variable
+ expansion syntax (e.g. $VAR for Linux and %VAR% for Windows).
+ <p>
+ Note: Use $$() method for cross-platform practice i.e. submit an
+ application from a Windows client to a Linux/Unix server or vice versa.
+ </p>]]>
+      </doc>
+    </method>
+    <method name="$$" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Expand the environment variable in platform-agnostic syntax. The
+ parameter expansion marker "{{VAR}}" will be replaced with real parameter
+ expansion marker ('%' for Windows and '$' for Linux) by NodeManager on
+ container launch. For example: {{VAR}} will be replaced as $VAR on Linux,
+ and %VAR% on Windows.]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[Environment for Applications.
+
+ Some of the environment variables for applications are <em>final</em>
+ i.e. they cannot be modified by the applications.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.api.ApplicationConstants.Environment -->
+  <!-- start interface org.apache.hadoop.yarn.api.ApplicationHistoryProtocol -->
+  <interface name="ApplicationHistoryProtocol"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.api.ApplicationBaseProtocol"/>
+    <doc>
+    <![CDATA[<p>
+ The protocol between clients and the <code>ApplicationHistoryServer</code> to
+ get the information of completed applications etc.
+ </p>]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.api.ApplicationHistoryProtocol -->
+  <!-- start interface org.apache.hadoop.yarn.api.ApplicationMasterProtocol -->
+  <interface name="ApplicationMasterProtocol"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="registerApplicationMaster" return="org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by a new <code>ApplicationMaster</code> to register with
+ the <code>ResourceManager</code>.
+ </p>
+
+ <p>
+ The <code>ApplicationMaster</code> needs to provide details such as RPC
+ Port, HTTP tracking url etc. as specified in
+ {@link RegisterApplicationMasterRequest}.
+ </p>
+
+ <p>
+ The <code>ResourceManager</code> responds with critical details such as
+ maximum resource capabilities in the cluster as specified in
+ {@link RegisterApplicationMasterResponse}.
+ </p>
+
+ @param request
+          registration request
+ @return registration respose
+ @throws YarnException
+ @throws IOException
+ @throws InvalidApplicationMasterRequestException
+           The exception is thrown when an ApplicationMaster tries to
+           register more then once.
+ @see RegisterApplicationMasterRequest
+ @see RegisterApplicationMasterResponse]]>
+      </doc>
+    </method>
+    <method name="finishApplicationMaster" return="org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by an <code>ApplicationMaster</code> to notify the
+ <code>ResourceManager</code> about its completion (success or failed).</p>
+
+ <p>The <code>ApplicationMaster</code> has to provide details such as
+ final state, diagnostics (in case of failures) etc. as specified in
+ {@link FinishApplicationMasterRequest}.</p>
+
+ <p>The <code>ResourceManager</code> responds with
+ {@link FinishApplicationMasterResponse}.</p>
+
+ @param request completion request
+ @return completion response
+ @throws YarnException
+ @throws IOException
+ @see FinishApplicationMasterRequest
+ @see FinishApplicationMasterResponse]]>
+      </doc>
+    </method>
+    <method name="allocate" return="org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The main interface between an <code>ApplicationMaster</code> and the
+ <code>ResourceManager</code>.
+ </p>
+
+ <p>
+ The <code>ApplicationMaster</code> uses this interface to provide a list of
+ {@link ResourceRequest} and returns unused {@link Container} allocated to
+ it via {@link AllocateRequest}. Optionally, the
+ <code>ApplicationMaster</code> can also <em>blacklist</em> resources which
+ it doesn't want to use.
+ </p>
+
+ <p>
+ This also doubles up as a <em>heartbeat</em> to let the
+ <code>ResourceManager</code> know that the <code>ApplicationMaster</code>
+ is alive. Thus, applications should periodically make this call to be kept
+ alive. The frequency depends on
+ {@link YarnConfiguration#RM_AM_EXPIRY_INTERVAL_MS} which defaults to
+ {@link YarnConfiguration#DEFAULT_RM_AM_EXPIRY_INTERVAL_MS}.
+ </p>
+
+ <p>
+ The <code>ResourceManager</code> responds with list of allocated
+ {@link Container}, status of completed containers and headroom information
+ for the application.
+ </p>
+
+ <p>
+ The <code>ApplicationMaster</code> can use the available headroom
+ (resources) to decide how to utilized allocated resources and make informed
+ decisions about future resource requests.
+ </p>
+
+ @param request
+          allocation request
+ @return allocation response
+ @throws YarnException
+ @throws IOException
+ @throws InvalidApplicationMasterRequestException
+           This exception is thrown when an ApplicationMaster calls allocate
+           without registering first.
+ @throws InvalidResourceBlacklistRequestException
+           This exception is thrown when an application provides an invalid
+           specification for blacklist of resources.
+ @throws InvalidResourceRequestException
+           This exception is thrown when a {@link ResourceRequest} is out of
+           the range of the configured lower and upper limits on the
+           resources.
+ @see AllocateRequest
+ @see AllocateResponse]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>The protocol between a live instance of <code>ApplicationMaster</code>
+ and the <code>ResourceManager</code>.</p>
+
+ <p>This is used by the <code>ApplicationMaster</code> to register/unregister
+ and to request and obtain resources in the cluster from the
+ <code>ResourceManager</code>.</p>]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.api.ApplicationMasterProtocol -->
+  <!-- start interface org.apache.hadoop.yarn.api.ClientSCMProtocol -->
+  <interface name="ClientSCMProtocol"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="use" return="org.apache.hadoop.yarn.api.protocolrecords.UseSharedCacheResourceResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.UseSharedCacheResourceRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients 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, the response is empty.
+ </p>
+
+ @param request request to claim a resource in the shared cache
+ @return response indicating if the resource is already in the cache
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="release" return="org.apache.hadoop.yarn.api.protocolrecords.ReleaseSharedCacheResourceResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReleaseSharedCacheResourceRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients 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>
+
+ <p>
+ Currently the <code>SharedCacheManager</code> sends an empty response.
+ </p>
+
+ @param request request to release a resource in the shared cache
+ @return (empty) response on releasing the resource
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>
+ The protocol between clients and the <code>SharedCacheManager</code> to claim
+ and release resources in the shared cache.
+ </p>]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.api.ClientSCMProtocol -->
+  <!-- start interface org.apache.hadoop.yarn.api.ClientSCMProtocolPB -->
+  <interface name="ClientSCMProtocolPB"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.proto.ClientSCMProtocol.ClientSCMProtocolService.BlockingInterface"/>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.api.ClientSCMProtocolPB -->
+  <!-- start interface org.apache.hadoop.yarn.api.ContainerManagementProtocol -->
+  <interface name="ContainerManagementProtocol"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="startContainers" return="org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The <code>ApplicationMaster</code> provides a list of
+ {@link StartContainerRequest}s to a <code>NodeManager</code> to
+ <em>start</em> {@link Container}s allocated to it using this interface.
+ </p>
+
+ <p>
+ The <code>ApplicationMaster</code> has to provide details such as allocated
+ resource capability, security tokens (if enabled), command to be executed
+ to start the container, environment for the process, necessary
+ binaries/jar/shared-objects etc. via the {@link ContainerLaunchContext} in
+ the {@link StartContainerRequest}.
+ </p>
+
+ <p>
+ The <code>NodeManager</code> sends a response via
+ {@link StartContainersResponse} which includes a list of
+ {@link Container}s of successfully launched {@link Container}s, a
+ containerId-to-exception map for each failed {@link StartContainerRequest} in
+ which the exception indicates errors from per container and a
+ allServicesMetaData map between the names of auxiliary services and their
+ corresponding meta-data. Note: None-container-specific exceptions will
+ still be thrown by the API method itself.
+ </p>
+ <p>
+ The <code>ApplicationMaster</code> can use
+ {@link #getContainerStatuses(GetContainerStatusesRequest)} to get updated
+ statuses of the to-be-launched or launched containers.
+ </p>
+
+ @param request
+          request to start a list of containers
+ @return response including conatinerIds of all successfully launched
+         containers, a containerId-to-exception map for failed requests and
+         a allServicesMetaData map.
+ @throws YarnException
+ @throws IOException
+ @throws NMNotYetReadyException
+           This exception is thrown when NM starts from scratch but has not
+           yet connected with RM.]]>
+      </doc>
+    </method>
+    <method name="stopContainers" return="org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The <code>ApplicationMaster</code> requests a <code>NodeManager</code> to
+ <em>stop</em> a list of {@link Container}s allocated to it using this
+ interface.
+ </p>
+
+ <p>
+ The <code>ApplicationMaster</code> sends a {@link StopContainersRequest}
+ which includes the {@link ContainerId}s of the containers to be stopped.
+ </p>
+
+ <p>
+ The <code>NodeManager</code> sends a response via
+ {@link StopContainersResponse} which includes a list of {@link ContainerId}
+ s of successfully stopped containers, a containerId-to-exception map for
+ each failed request in which the exception indicates errors from per
+ container. Note: None-container-specific exceptions will still be thrown by
+ the API method itself. <code>ApplicationMaster</code> can use
+ {@link #getContainerStatuses(GetContainerStatusesRequest)} to get updated
+ statuses of the containers.
+ </p>
+
+ @param request
+          request to stop a list of containers
+ @return response which includes a list of containerIds of successfully
+         stopped containers, a containerId-to-exception map for failed
+         requests.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainerStatuses" return="org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The API used by the <code>ApplicationMaster</code> to request for current
+ statuses of <code>Container</code>s from the <code>NodeManager</code>.
+ </p>
+
+ <p>
+ The <code>ApplicationMaster</code> sends a
+ {@link GetContainerStatusesRequest} which includes the {@link ContainerId}s
+ of all containers whose statuses are needed.
+ </p>
+
+ <p>
+ The <code>NodeManager</code> responds with
+ {@link GetContainerStatusesResponse} which includes a list of
+ {@link ContainerStatus} of the successfully queried containers and a
+ containerId-to-exception map for each failed request in which the exception
+ indicates errors from per container. Note: None-container-specific
+ exceptions will still be thrown by the API method itself.
+ </p>
+
+ @param request
+          request to get <code>ContainerStatus</code>es of containers with
+          the specified <code>ContainerId</code>s
+ @return response containing the list of <code>ContainerStatus</code> of the
+         successfully queried containers and a containerId-to-exception map
+         for failed requests.
+
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>The protocol between an <code>ApplicationMaster</code> and a
+ <code>NodeManager</code> to start/stop containers and to get status
+ of running containers.</p>
+
+ <p>If security is enabled the <code>NodeManager</code> verifies that the
+ <code>ApplicationMaster</code> has truly been allocated the container
+ by the <code>ResourceManager</code> and also verifies all interactions such
+ as stopping the container or obtaining status information for the container.
+ </p>]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.api.ContainerManagementProtocol -->
+</package>
+<package name="org.apache.hadoop.yarn.api.protocolrecords">
+  <!-- start class org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest -->
+  <class name="AllocateRequest" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AllocateRequest"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="newInstance" return="org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="responseID" type="int"/>
+      <param name="appProgress" type="float"/>
+      <param name="resourceAsk" type="java.util.List"/>
+      <param name="containersToBeReleased" type="java.util.List"/>
+      <param name="resourceBlacklistRequest" type="org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest"/>
+    </method>
+    <method name="newInstance" return="org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="responseID" type="int"/>
+      <param name="appProgress" type="float"/>
+      <param name="resourceAsk" type="java.util.List"/>
+      <param name="containersToBeReleased" type="java.util.List"/>
+      <param name="resourceBlacklistRequest" type="org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest"/>
+      <param name="increaseRequests" type="java.util.List"/>
+    </method>
+    <method name="getResponseId" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <em>response id</em> used to track duplicate responses.
+ @return <em>response id</em>]]>
+      </doc>
+    </method>
+    <method name="setResponseId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="id" type="int"/>
+      <doc>
+      <![CDATA[Set the <em>response id</em> used to track duplicate responses.
+ @param id <em>response id</em>]]>
+      </doc>
+    </method>
+    <method name="getProgress" return="float"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <em>current progress</em> of application.
+ @return <em>current progress</em> of application]]>
+      </doc>
+    </method>
+    <method name="setProgress"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="progress" type="float"/>
+      <doc>
+      <![CDATA[Set the <em>current progress</em> of application
+ @param progress <em>current progress</em> of application]]>
+      </doc>
+    </method>
+    <method name="getAskList" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the list of <code>ResourceRequest</code> to update the
+ <code>ResourceManager</code> about the application's resource requirements.
+ @return the list of <code>ResourceRequest</code>
+ @see ResourceRequest]]>
+      </doc>
+    </method>
+    <method name="setAskList"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="resourceRequests" type="java.util.List"/>
+      <doc>
+      <![CDATA[Set list of <code>ResourceRequest</code> to update the
+ <code>ResourceManager</code> about the application's resource requirements.
+ @param resourceRequests list of <code>ResourceRequest</code> to update the
+                        <code>ResourceManager</code> about the application's
+                        resource requirements
+ @see ResourceRequest]]>
+      </doc>
+    </method>
+    <method name="getReleaseList" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the list of <code>ContainerId</code> of containers being
+ released by the <code>ApplicationMaster</code>.
+ @return list of <code>ContainerId</code> of containers being
+         released by the <code>ApplicationMaster</code>]]>
+      </doc>
+    </method>
+    <method name="setReleaseList"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="releaseContainers" type="java.util.List"/>
+      <doc>
+      <![CDATA[Set the list of <code>ContainerId</code> of containers being
+ released by the <code>ApplicationMaster</code>
+ @param releaseContainers list of <code>ContainerId</code> of
+                          containers being released by the
+                          <code>ApplicationMaster</code>]]>
+      </doc>
+    </method>
+    <method name="getResourceBlacklistRequest" return="org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <code>ResourceBlacklistRequest</code> being sent by the
+ <code>ApplicationMaster</code>.
+ @return the <code>ResourceBlacklistRequest</code> being sent by the
+         <code>ApplicationMaster</code>
+ @see ResourceBlacklistRequest]]>
+      </doc>
+    </method>
+    <method name="setResourceBlacklistRequest"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="resourceBlacklistRequest" type="org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest"/>
+      <doc>
+      <![CDATA[Set the <code>ResourceBlacklistRequest</code> to inform the
+ <code>ResourceManager</code> about the blacklist additions and removals
+ per the <code>ApplicationMaster</code>.
+
+ @param resourceBlacklistRequest the <code>ResourceBlacklistRequest</code>
+                         to inform the <code>ResourceManager</code> about
+                         the blacklist additions and removals
+                         per the <code>ApplicationMaster</code>
+ @see ResourceBlacklistRequest]]>
+      </doc>
+    </method>
+    <method name="getIncreaseRequests" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <code>ContainerResourceIncreaseRequest</code> being sent by the
+ <code>ApplicationMaster</code>]]>
+      </doc>
+    </method>
+    <method name="setIncreaseRequests"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="increaseRequests" type="java.util.List"/>
+      <doc>
+      <![CDATA[Set the <code>ContainerResourceIncreaseRequest</code> to inform the
+ <code>ResourceManager</code> about some container's resources need to be
+ increased]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>The core request sent by the <code>ApplicationMaster</code> to the
+ <code>ResourceManager</code> to obtain resources in the cluster.</p>
+
+ <p>The request includes:
+ <ul>
+   <li>A response id to track duplicate responses.</li>
+   <li>Progress information.</li>
+   <li>
+     A list of {@link ResourceRequest} to inform the
+     <code>ResourceManager</code> about the application's
+     resource requirements.
+   </li>
+   <li>
+     A list of unused {@link Container} which are being returned.
+   </li>
+ </ul>
+
+ @see ApplicationMasterProtocol#allocate(AllocateRequest)]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest -->
+  <!-- start class org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse -->
+  <class name="AllocateResponse" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AllocateResponse"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="newInstance" return="org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="responseId" type="int"/>
+      <param name="completedContainers" type="java.util.List"/>
+      <param name="allocatedContainers" type="java.util.List"/>
+      <param name="updatedNodes" type="java.util.List"/>
+      <param name="availResources" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <param name="command" type="org.apache.hadoop.yarn.api.records.AMCommand"/>
+      <param name="numClusterNodes" type="int"/>
+      <param name="preempt" type="org.apache.hadoop.yarn.api.records.PreemptionMessage"/>
+      <param name="nmTokens" type="java.util.List"/>
+    </method>
+    <method name="newInstance" return="org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="responseId" type="int"/>
+      <param name="completedContainers" type="java.util.List"/>
+      <param name="allocatedContainers" type="java.util.List"/>
+      <param name="updatedNodes" type="java.util.List"/>
+      <param name="availResources" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <param name="command" type="org.apache.hadoop.yarn.api.records.AMCommand"/>
+      <param name="numClusterNodes" type="int"/>
+      <param name="preempt" type="org.apache.hadoop.yarn.api.records.PreemptionMessage"/>
+      <param name="nmTokens" type="java.util.List"/>
+      <param name="increasedContainers" type="java.util.List"/>
+      <param name="decreasedContainers" type="java.util.List"/>
+    </method>
+    <method name="getAMCommand" return="org.apache.hadoop.yarn.api.records.AMCommand"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[If the <code>ResourceManager</code> needs the
+ <code>ApplicationMaster</code> to take some action then it will send an
+ AMCommand to the <code>ApplicationMaster</code>. See <code>AMCommand</code>
+ for details on commands and actions for them.
+ @return <code>AMCommand</code> if the <code>ApplicationMaster</code> should
+         take action, <code>null</code> otherwise
+ @see AMCommand]]>
+      </doc>
+    </method>
+    <method name="getResponseId" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <em>last response id</em>.
+ @return <em>last response id</em>]]>
+      </doc>
+    </method>
+    <method name="getAllocatedContainers" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the list of <em>newly allocated</em> <code>Container</code> by the
+ <code>ResourceManager</code>.
+ @return list of <em>newly allocated</em> <code>Container</code>]]>
+      </doc>
+    </method>
+    <method name="getAvailableResources" return="org.apache.hadoop.yarn.api.records.Resource"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <em>available headroom</em> for resources in the cluster for the
+ application.
+ @return limit of available headroom for resources in the cluster for the
+ application]]>
+      </doc>
+    </method>
+    <method name="getCompletedContainersStatuses" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the list of <em>completed containers' statuses</em>.
+ @return the list of <em>completed containers' statuses</em>]]>
+      </doc>
+    </method>
+    <method name="getUpdatedNodes" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the list of <em>updated <code>NodeReport</code>s</em>. Updates could
+ be changes in health, availability etc of the nodes.
+ @return The delta of updated nodes since the last response]]>
+      </doc>
+    </method>
+    <method name="getNumClusterNodes" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the number of hosts available on the cluster.
+ @return the available host count.]]>
+      </doc>
+    </method>
+    <method name="getPreemptionMessage" return="org.apache.hadoop.yarn.api.records.PreemptionMessage"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the description of containers owned by the AM, but requested back by
+ the cluster. Note that the RM may have an inconsistent view of the
+ resources owned by the AM. These messages are advisory, and the AM may
+ elect to ignore them.
+ <p>
+ The message is a snapshot of the resources the RM wants back from the AM.
+ While demand persists, the RM will repeat its request; applications should
+ not interpret each message as a request for <em>additional</em>
+ resources on top of previous messages. Resources requested consistently
+ over some duration may be forcibly killed by the RM.
+
+ @return A specification of the resources to reclaim from this AM.]]>
+      </doc>
+    </method>
+    <method name="getNMTokens" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the list of NMTokens required for communicating with NM. New NMTokens
+ issued only if
+ <p>
+ 1) AM is receiving first container on underlying NodeManager.<br>
+ OR<br>
+ 2) NMToken master key rolled over in ResourceManager and AM is getting new
+ container on the same underlying NodeManager.
+ <p>
+ AM will receive one NMToken per NM irrespective of the number of containers
+ issued on same NM. AM is expected to store these tokens until issued a
+ new token for the same NM.]]>
+      </doc>
+    </method>
+    <method name="getIncreasedContainers" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the list of newly increased containers by <code>ResourceManager</code>]]>
+      </doc>
+    </method>
+    <method name="getDecreasedContainers" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the list of newly decreased containers by <code>NodeManager</code>]]>
+      </doc>
+    </method>
+    <method name="getAMRMToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[The AMRMToken that belong to this attempt
+
+ @return The AMRMToken that belong to this attempt]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[The response sent by the <code>ResourceManager</code> the
+ <code>ApplicationMaster</code> during resource negotiation.
+ <p>
+ The response, includes:
+ <ul>
+   <li>Response ID to track duplicate responses.</li>
+   <li>
+     An AMCommand sent by ResourceManager to let the
+     {@code ApplicationMaster} take some actions (resync, shutdown etc.).
+   </li>
+   <li>A list of newly allocated {@link Container}.</li>
+   <li>A list of completed {@link Container}s' statuses.</li>
+   <li>
+     The available headroom for resources in the cluster for the
+     application.
+   </li>
+   <li>A list of nodes whose status has been updated.</li>
+   <li>The number of available nodes in a cluster.</li>
+   <li>A description of resources requested back by the cluster</li>
+   <li>AMRMToken, if AMRMToken has been rolled over</li>
+ </ul>
+
+ @see ApplicationMasterProtocol#allocate(AllocateRequest)]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse -->
+  <!-- start class org.apache.hadoop.yarn.api.protocolrecords.ApplicationsRequestScope -->
+  <class name="ApplicationsRequestScope" extends="java.lang.Enum"
+    abstract="false"
+    static="false" final="true" visibility="public"
+    deprecated="not deprecated">
+    <method name="values" return="org.apache.hadoop.yarn.api.protocolrecords.ApplicationsRequestScope[]"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="valueOf" return="org.apache.hadoop.yarn.api.protocolrecords.ApplicationsRequestScope"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+    </method>
+    <doc>
+    <![CDATA[Enumeration that controls the scope of applications fetched]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.api.protocolrecords.ApplicationsRequestScope -->
+  <!-- start class org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest -->
+  <class name="FinishApplicationMasterRequest" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="FinishApplicationMasterRequest"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="newInstance" return="org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="finalAppStatus" type="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"/>
+      <param name="diagnostics" type="java.lang.String"/>
+      <param name="url" type="java.lang.String"/>
+    </method>
+    <method name="getFinalApplicationStatus" return="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get <em>final state</em> of the <code>ApplicationMaster</code>.
+ @return <em>final state</em> of the <code>ApplicationMaster</code>]]>
+      </doc>
+    </method>
+    <method name="setFinalApplicationStatus"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="finalState" type="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"/>
+      <doc>
+      <![CDATA[Set the <em>final state</em> of the <code>ApplicationMaster</code>
+ @param finalState <em>final state</em> of the <code>ApplicationMaster</code>]]>
+      </doc>
+    </method>
+    <method name="getDiagnostics" return="java.lang.String"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get <em>diagnostic information</em> on application failure.
+ @return <em>diagnostic information</em> on application failure]]>
+      </doc>
+    </method>
+    <method name="setDiagnostics"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="diagnostics" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Set <em>diagnostic information</em> on application failure.
+ @param diagnostics <em>diagnostic information</em> on application failure]]>
+      </doc>
+    </method>
+    <method name="getTrackingUrl" return="java.lang.String"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <em>tracking URL</em> for the <code>ApplicationMaster</code>.
+ This url if contains scheme then that will be used by resource manager
+ web application proxy otherwise it will default to http.
+ @return <em>tracking URL</em>for the <code>ApplicationMaster</code>]]>
+      </doc>
+    </method>
+    <method name="setTrackingUrl"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="url" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Set the <em>final tracking URL</em>for the <code>ApplicationMaster</code>.
+ This is the web-URL to which ResourceManager or web-application proxy will
+ redirect client/users once the application is finished and the
+ <code>ApplicationMaster</code> is gone.
+ <p>
+ If the passed url has a scheme then that will be used by the
+ ResourceManager and web-application proxy, otherwise the scheme will
+ default to http.
+ </p>
+ <p>
+ Empty, null, "N/A" strings are all valid besides a real URL. In case an url
+ isn't explicitly passed, it defaults to "N/A" on the ResourceManager.
+ <p>
+
+ @param url
+          <em>tracking URL</em>for the <code>ApplicationMaster</code>]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[The finalization request sent by the {@code ApplicationMaster} to
+ inform the {@code ResourceManager} about its completion.
+ <p>
+ The final request includes details such:
+ <ul>
+   <li>Final state of the {@code ApplicationMaster}</li>
+   <li>
+     Diagnostic information in case of failure of the
+     {@code ApplicationMaster}
+   </li>
+   <li>Tracking URL</li>
+ </ul>
+
+ @see ApplicationMasterProtocol#finishApplicationMaster(FinishApplicationMasterRequest)]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest -->
+  <!-- start class org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse -->
+  <class name="FinishApplicationMasterResponse" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="FinishApplicationMasterResponse"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getIsUnregistered" return="boolean"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the flag which indicates that the application has successfully
+ unregistered with the RM and the application can safely stop.]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[The response sent by the <code>ResourceManager</code> to a
+ <code>ApplicationMaster</code> on it's completion.
+ <p>
+ The response, includes:
+ <ul>
+ <li>A flag which indicates that the application has successfully unregistered
+ with the RM and the application can safely stop.</li>
+ </ul>
+ <p>
+ Note: The flag indicates whether the application has successfully
+ unregistered and is safe to stop. The application may stop after the flag is
+ true. If the application stops before the flag is true then the RM may retry
+ the application.
+
+ @see ApplicationMasterProtocol#finishApplicationMaster(FinishApplicationMasterRequest)]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse -->
+  <!-- start class org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest -->
+  <class name="GetApplicationAttemptReportRequest" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="GetApplicationAttemptReportRequest"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="newInstance" return="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+    </method>
+    <method name="getApplicationAttemptId" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <code>ApplicationAttemptId</code> of an application attempt.
+
+ @return <code>ApplicationAttemptId</code> of an application attempt]]>
+      </doc>
+    </method>
+    <method name="setApplicationAttemptId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <doc>
+      <![CDATA[Set the <code>ApplicationAttemptId</code> of an application attempt
+
+ @param applicationAttemptId
+          <code>ApplicationAttemptId</code> of an application attempt]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>
+ The request sent by a client to the <code>ResourceManager</code> to get an
+ {@link ApplicationAttemptReport} for an application attempt.
+ </p>
+
+ <p>
+ The request should include the {@link ApplicationAttemptId} of the
+ application attempt.
+ </p>
+
+ @see ApplicationAttemptReport
+ @see ApplicationHistoryProtocol#getApplicationAttemptReport(GetApplicationAttemptReportRequest)]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest -->
+  <!-- start class org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse -->
+  <class name="GetApplicationAttemptReportResponse" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="GetApplicationAttemptReportResponse"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="newInstance" return="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="ApplicationAttemptReport" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptReport"/>
+    </method>
+    <method name="getApplicationAttemptReport" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <code>ApplicationAttemptReport</code> for the application attempt.
+
+ @return <code>ApplicationAttemptReport</code> for the application attempt]]>
+      </doc>
+    </method>
+    <method name="setApplicationAttemptReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptReport" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptReport"/>
+      <doc>
+      <![CDATA[Get the <code>ApplicationAttemptReport</code> for the application attempt.
+
+ @param applicationAttemptReport
+          <code>ApplicationAttemptReport</code> for the application attempt]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>
+ The response sent by the <code>ResourceManager</code> to a client requesting
+ an application attempt report.
+ </p>
+
+ <p>
+ The response includes an {@link ApplicationAttemptReport} which has the
+ details about the particular application attempt
+ </p>
+
+ @see ApplicationAttemptReport
+ @see ApplicationHistoryProtocol#getApplicationAttemptReport(GetApplicationAttemptReportRequest)]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse -->
+  <!-- start class org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest -->
+  <class name="GetApplicationAttemptsRequest" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="GetApplicationAttemptsRequest"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="newInstance" return="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+    </method>
+    <method name="getApplicationId" return="org.apache.hadoop.yarn.api.records.ApplicationId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <code>ApplicationId</code> of an application
+
+ @return <code>ApplicationId</code> of an application]]>
+      </doc>
+    </method>
+    <method name="setApplicationId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <doc>
+      <![CDATA[Set the <code>ApplicationId</code> of an application
+
+ @param applicationId
+          <code>ApplicationId</code> of an application]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>
+ The request from clients to get a list of application attempt reports of an
+ application from the <code>ResourceManager</code>.
+ </p>
+
+ @see ApplicationHistoryProtocol#getApplicationAttempts(GetApplicationAttemptsRequest)]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest -->
+  <!-- start class org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse -->
+  <class name="GetApplicationAttemptsResponse" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="GetApplicationAttemptsResponse"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="newInstance" return="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttempts" type="java.util.List"/>
+    </method>
+    <method name="getApplicationAttemptList" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get a list of <code>ApplicationReport</code> of an application.
+
+ @return a list of <code>ApplicationReport</code> of an application]]>
+      </doc>
+    </method>
+    <method name="setApplicationAttemptList"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttempts" type="java.util.List"/>
+      <doc>
+      <![CDATA[Get a list of <code>ApplicationReport</code> of an application.
+
+ @param applicationAttempts
+          a list of <code>ApplicationReport</code> of an application]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>
+ The response sent by the <code>ResourceManager</code> to a client requesting
+ a list of {@link ApplicationAttemptReport} for application attempts.
+ </p>
+
+ <p>
+ The <code>ApplicationAttemptReport</code> for each application includes the
+ details of an application attempt.
+ </p>
+
+ @see ApplicationAttemptReport
+ @see ApplicationHistoryProtocol#getApplicationAttempts(GetApplicationAttemptsRequest)]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse -->
+  <!-- start class org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest -->
+  <class name="GetApplicationReportRequest" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="GetApplicationReportRequest"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="newInstance" return="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+    </method>
+    <method name="getApplicationId" return="org.apache.hadoop.yarn.api.records.ApplicationId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <code>ApplicationId</code> of the application.
+ @return <code>ApplicationId</code> of the application]]>
+      </doc>
+    </method>
+    <method name="setApplicationId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <doc>
+      <![CDATA[Set the <code>ApplicationId</code> of the application
+ @param applicationId <code>ApplicationId</code> of the application]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>The request sent by a client to the <code>ResourceManager</code> to
+ get an {@link ApplicationReport} for an application.</p>
+
+ <p>The request should include the {@link ApplicationId} of the
+ application.</p>
+
+ @see ApplicationClientProtocol#getApplicationReport(GetApplicationReportRequest)
+ @see ApplicationReport]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest -->
+  <!-- start class org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse -->
+  <class name="GetApplicationReportResponse" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="GetApplicationReportResponse"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getApplicationReport" return="org.apache.hadoop.yarn.api.records.ApplicationReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <code>ApplicationReport</code> for the application.
+ @return <code>ApplicationReport</code> for the application]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>The response sent by the <code>ResourceManager</code> to a client
+ requesting an application report.</p>
+
+ <p>The response includes an {@link ApplicationReport} which has details such
+ as user, queue, name, host on which the <code>ApplicationMaster</code> is
+ running, RPC port, tracking URL, diagnostics, start time etc.</p>
+
+ @see ApplicationClientProtocol#getApplicationReport(GetApplicationReportRequest)]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse -->
+  <!-- start class org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest -->
+  <class name="GetApplicationsRequest" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="GetApplicationsRequest"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="newInstance" return="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="newInstance" return="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="scope" type="org.apache.hadoop.yarn.api.protocolrecords.ApplicationsRequestScope"/>
+      <param name="users" type="java.util.Set"/>
+      <param name="queues" type="java.util.Set"/>
+      <param name="applicationTypes" type="java.util.Set"/>
+      <param name="applicationTags" type="java.util.Set"/>
+      <param name="appli

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[18/51] [abbrv] hadoop git commit: HADOOP-12537 S3A to support Amazon STS temporary credentials. Contributed by Sean Mackrory.

Posted by vv...@apache.org.
HADOOP-12537 S3A to support Amazon STS temporary credentials. Contributed by Sean Mackrory.


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

Branch: refs/heads/YARN-3926
Commit: 31ffaf76f2b6e1fd2a141daa4daaebdfecefe727
Parents: 9378d94
Author: Steve Loughran <st...@apache.org>
Authored: Thu Jun 9 20:58:30 2016 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Jun 9 21:00:47 2016 +0100

----------------------------------------------------------------------
 .../src/main/resources/core-default.xml         |   5 +
 hadoop-project/pom.xml                          |   8 +-
 hadoop-tools/hadoop-aws/pom.xml                 |   5 +
 .../fs/s3a/BasicAWSCredentialsProvider.java     |   3 +-
 .../org/apache/hadoop/fs/s3a/Constants.java     |   3 +
 .../s3a/CredentialInitializationException.java  |  46 ++++++
 .../fs/s3a/TemporaryAWSCredentialsProvider.java |  70 +++++++++
 .../src/site/markdown/tools/hadoop-aws/index.md |  71 ++++++++-
 .../fs/s3a/TestS3ATemporaryCredentials.java     | 150 +++++++++++++++++++
 9 files changed, 357 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 8bb27ea..39b7132 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -806,6 +806,11 @@
 </property>
 
 <property>
+  <name>fs.s3a.session.token</name>
+  <description>The session token used with temporary credentials. Used only with provider org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider.</description>
+</property>
+
+<property>
   <name>fs.s3a.connection.maximum</name>
   <value>15</value>
   <description>Controls the maximum number of simultaneous connections to S3.</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 2b6b162..4c618a1 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -116,6 +116,7 @@
     <make-maven-plugin.version>1.0-beta-1</make-maven-plugin.version>
     <native-maven-plugin.version>1.0-alpha-8</native-maven-plugin.version>
     <surefire.fork.timeout>900</surefire.fork.timeout>
+    <aws-java-sdk.version>1.10.6</aws-java-sdk.version>
   </properties>
 
   <dependencyManagement>
@@ -690,7 +691,12 @@
       <dependency>
         <groupId>com.amazonaws</groupId>
         <artifactId>aws-java-sdk-s3</artifactId>
-        <version>1.10.6</version>
+        <version>${aws-java-sdk.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>com.amazonaws</groupId>
+        <artifactId>aws-java-sdk-sts</artifactId>
+        <version>${aws-java-sdk.version}</version>
       </dependency>
       <dependency>
         <groupId>org.apache.mina</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/hadoop-tools/hadoop-aws/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml
index c95f1e6..7c25e60 100644
--- a/hadoop-tools/hadoop-aws/pom.xml
+++ b/hadoop-tools/hadoop-aws/pom.xml
@@ -231,6 +231,11 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
+      <groupId>com.amazonaws</groupId>
+      <artifactId>aws-java-sdk-sts</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
index 3a5ee8c..61be43f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.fs.s3a;
 
-import com.amazonaws.AmazonClientException;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.BasicAWSCredentials;
 import com.amazonaws.auth.AWSCredentials;
@@ -49,7 +48,7 @@ public class BasicAWSCredentialsProvider implements AWSCredentialsProvider {
     if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)) {
       return new BasicAWSCredentials(accessKey, secretKey);
     }
-    throw new AmazonClientException(
+    throw new CredentialInitializationException(
         "Access key or secret key is null");
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/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 eb859ac..4abb550 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
@@ -41,6 +41,9 @@ public final class Constants {
   public static final String AWS_CREDENTIALS_PROVIDER =
       "fs.s3a.aws.credentials.provider";
 
+  // session token for when using TemporaryAWSCredentialsProvider
+  public static final String SESSION_TOKEN = "fs.s3a.session.token";
+
   // number of simultaneous connections to s3
   public static final String MAXIMUM_CONNECTIONS = "fs.s3a.connection.maximum";
   public static final int DEFAULT_MAXIMUM_CONNECTIONS = 15;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java
new file mode 100644
index 0000000..46655bc
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/CredentialInitializationException.java
@@ -0,0 +1,46 @@
+/*
+ * 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.AmazonClientException;
+
+/**
+ * Exception which Hadoop's AWSCredentialsProvider implementations should
+ * throw when there is a problem with the credential setup. This
+ * is a subclass of {@link AmazonClientException} which sets
+ * {@link #isRetryable()} to false, so as to fail fast.
+ */
+public class CredentialInitializationException extends AmazonClientException {
+  public CredentialInitializationException(String message, Throwable t) {
+    super(message, t);
+  }
+
+  public CredentialInitializationException(String message) {
+    super(message);
+  }
+
+  /**
+   * This exception is not going to go away if you try calling it again.
+   * @return false, always.
+   */
+  @Override
+  public boolean isRetryable() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java
new file mode 100644
index 0000000..190f7bc
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java
@@ -0,0 +1,70 @@
+/**
+ * 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.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.BasicSessionCredentials;
+import com.amazonaws.auth.AWSCredentials;
+import org.apache.commons.lang.StringUtils;
+
+import java.net.URI;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
+/**
+ * Support session credentials for authenticating with AWS.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class TemporaryAWSCredentialsProvider implements AWSCredentialsProvider {
+
+  public static final String NAME
+      = "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
+  private final String accessKey;
+  private final String secretKey;
+  private final String sessionToken;
+
+  public TemporaryAWSCredentialsProvider(URI uri, Configuration conf) {
+    this.accessKey = conf.get(ACCESS_KEY, null);
+    this.secretKey = conf.get(SECRET_KEY, null);
+    this.sessionToken = conf.get(SESSION_TOKEN, null);
+  }
+
+  public AWSCredentials getCredentials() {
+    if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)
+        && !StringUtils.isEmpty(sessionToken)) {
+      return new BasicSessionCredentials(accessKey, secretKey, sessionToken);
+    }
+    throw new CredentialInitializationException(
+        "Access key, secret key or session token is unset");
+  }
+
+  @Override
+  public void refresh() {}
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index 4086bc0..606275c 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -201,6 +201,46 @@ If you do any of these: change your credentials immediately!
       </description>
     </property>
 
+    <property>
+      <name>fs.s3a.session.token</name>
+      <description>Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider as the providers.</description>
+    </property>
+
+#### Authentication methods
+
+The standard way to authenticate is with an access key and secret key using the
+properties above. You can also avoid configuring credentials if the EC2
+instances in your cluster are configured with IAM instance profiles that grant
+the appropriate S3 access.
+
+A temporary set of credentials can also be obtained from Amazon STS; these
+consist of an access key, a secret key, and a session token. To use these
+temporary credentials you must include the `aws-java-sdk-sts` JAR in your
+classpath (consult the POM for the current version) and set the
+`TemporaryAWSCredentialsProvider` class as the provider. The session key
+must be set in the property `fs.s3a.session.token` \u2014and the access and secret
+key properties to those of this temporary session.
+
+    <property>
+      <name>fs.s3a.aws.credentials.provider</name>
+      <value>org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider</value>
+    </property>
+
+    <property>
+      <name>fs.s3a.access.key</name>
+      <value>SESSION-ACCESS-KEY</value>
+    </property>
+
+    <property>
+      <name>fs.s3a.secret.key</name>
+      <value>SESSION-SECRET-KEY</value>
+    </property>
+
+    <property>
+      <name>fs.s3a.session.token</name>
+      <value>SECRET-SESSION-TOKEN</value>
+    </property>
+
 #### Protecting the AWS Credentials in S3A
 
 To protect the access/secret keys from prying eyes, it is recommended that you
@@ -605,6 +645,13 @@ Example:
         <description>AWS secret key. Omit for IAM role-based authentication.</description>
         <value>DONOTEVERSHARETHISSECRETKEY!</value>
       </property>
+
+      <property>
+        <name>test.sts.endpoint</name>
+        <description>Specific endpoint to use for STS requests.</description>
+        <value>sts.amazonaws.com</value>
+      </property>
+
     </configuration>
 
 ### File `contract-test-options.xml`
@@ -714,8 +761,30 @@ that the file `contract-test-options.xml` does not contain any
 secret credentials itself. As the auth keys XML file is kept out of the
 source code tree, it is not going to get accidentally committed.
 
-### Running Performance Tests against non-AWS storage infrastructures
+### Running Tests against non-AWS storage infrastructures
+
+### S3A session tests
+
+The test `TestS3ATemporaryCredentials` requests a set of temporary
+credentials from the STS service, then uses them to authenticate with S3.
+
+If an S3 implementation does not support STS, then the functional test
+cases must be disabled:
+
+        <property>
+          <name>test.fs.s3a.sts.enabled</name>
+          <value>false</value>
+        </property>
+
+These tests reqest a temporary set of credentials from the STS service endpoint.
+An alternate endpoint may be defined in `test.fs.s3a.sts.endpoint`.
+
+        <property>
+          <name>test.fs.s3a.sts.endpoint</name>
+          <value>https://sts.example.org/</value>
+        </property>
 
+The default is ""; meaning "use the amazon default value".
 
 #### CSV Data source
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/31ffaf76/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ATemporaryCredentials.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ATemporaryCredentials.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ATemporaryCredentials.java
new file mode 100644
index 0000000..be5c599
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ATemporaryCredentials.java
@@ -0,0 +1,150 @@
+/**
+ * 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 java.io.IOException;
+
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProviderChain;
+import com.amazonaws.auth.InstanceProfileCredentialsProvider;
+import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient;
+import com.amazonaws.services.securitytoken.model.GetSessionTokenRequest;
+import com.amazonaws.services.securitytoken.model.GetSessionTokenResult;
+import com.amazonaws.services.securitytoken.model.Credentials;
+
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
+import org.apache.hadoop.conf.Configuration;
+
+import org.junit.Test;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
+/**
+ * Tests use of temporary credentials (for example, AWS STS & S3).
+ * This test extends a class that "does things to the root directory", and
+ * should only be used against transient filesystems where you don't care about
+ * the data.
+ */
+public class TestS3ATemporaryCredentials extends AbstractFSContractTestBase {
+  public static final String TEST_STS_ENABLED = "test.fs.s3a.sts.enabled";
+  public static final String TEST_STS_ENDPOINT = "test.fs.s3a.sts.endpoint";
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestS3ATemporaryCredentials.class);
+
+  private S3AFileSystem fs;
+
+
+  private static final String PROVIDER_CLASS =
+      "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
+
+  private static final long TEST_FILE_SIZE = 1024;
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new S3AContract(conf);
+  }
+
+  /**
+   * Test use of STS for requesting temporary credentials.
+   *
+   * The property test.sts.endpoint can be set to point this at different
+   * STS endpoints. This test will use the AWS credentials (if provided) for
+   * S3A tests to request temporary credentials, then attempt to use those
+   * credentials instead.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testSTS() throws IOException {
+    Configuration conf = getContract().getConf();
+    if (!conf.getBoolean(TEST_STS_ENABLED, true)) {
+      skip("STS functional tests disabled");
+    }
+
+    String parentAccessKey = conf.getTrimmed(ACCESS_KEY, null);
+    String parentSecretKey = conf.getTrimmed(SECRET_KEY, null);
+    String stsEndpoint = conf.getTrimmed(TEST_STS_ENDPOINT, "");
+    AWSCredentialsProviderChain parentCredentials;
+    parentCredentials = new AWSCredentialsProviderChain(
+      new BasicAWSCredentialsProvider(parentAccessKey, parentSecretKey),
+      new InstanceProfileCredentialsProvider()
+    );
+
+    AWSSecurityTokenServiceClient stsClient;
+    stsClient = new AWSSecurityTokenServiceClient(parentCredentials);
+    if (!stsEndpoint.isEmpty()) {
+      LOG.debug("STS Endpoint ={}", stsEndpoint);
+      stsClient.setEndpoint(stsEndpoint);
+    }
+    GetSessionTokenRequest sessionTokenRequest = new GetSessionTokenRequest();
+    sessionTokenRequest.setDurationSeconds(900);
+    GetSessionTokenResult sessionTokenResult;
+    sessionTokenResult = stsClient.getSessionToken(sessionTokenRequest);
+    Credentials sessionCreds = sessionTokenResult.getCredentials();
+
+    String childAccessKey = sessionCreds.getAccessKeyId();
+    conf.set(ACCESS_KEY, childAccessKey);
+    String childSecretKey = sessionCreds.getSecretAccessKey();
+    conf.set(SECRET_KEY, childSecretKey);
+    String sessionToken = sessionCreds.getSessionToken();
+    conf.set(SESSION_TOKEN, sessionToken);
+
+    conf.set(AWS_CREDENTIALS_PROVIDER, PROVIDER_CLASS);
+
+    try(S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
+      createAndVerifyFile(fs, path("testSTS"), TEST_FILE_SIZE);
+    }
+
+    // now create an invalid set of credentials by changing the session
+    // token
+    conf.set(SESSION_TOKEN, "invalid-" + sessionToken);
+    try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
+      createAndVerifyFile(fs, path("testSTSInvalidToken"), TEST_FILE_SIZE);
+      fail("Expected an access exception, but file access to "
+          + fs.getUri() + " was allowed: " + fs);
+    } catch (AWSS3IOException ex) {
+      LOG.info("Expected Exception: {}", ex.toString());
+      LOG.debug("Expected Exception: {}", ex, ex);
+    }
+  }
+
+  @Test
+  public void testTemporaryCredentialValidation() throws Throwable {
+    Configuration conf = new Configuration();
+    conf.set(ACCESS_KEY, "accesskey");
+    conf.set(SECRET_KEY, "secretkey");
+    conf.set(SESSION_TOKEN, "");
+    TemporaryAWSCredentialsProvider provider
+        = new TemporaryAWSCredentialsProvider(getFileSystem().getUri(), conf);
+    try {
+      AWSCredentials credentials = provider.getCredentials();
+      fail("Expected a CredentialInitializationException,"
+          + " got " + credentials);
+    } catch (CredentialInitializationException expected) {
+      // expected
+    }
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[09/51] [abbrv] hadoop git commit: YARN-5080. Addendum fix to the original patch to fix YARN logs CLI. Contributed by Xuan Gong

Posted by vv...@apache.org.
YARN-5080. Addendum fix to the original patch to fix YARN logs CLI. Contributed by Xuan Gong


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

Branch: refs/heads/YARN-3926
Commit: 5a43583c0bbb9650ea6a9f48d9544ec3ec24b580
Parents: 3344ba7
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Wed Jun 8 09:49:55 2016 -0700
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Wed Jun 8 09:49:55 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a43583c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
index bbe636f..d62ee5e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
@@ -278,7 +278,9 @@ public class LogsCLI extends Configured implements Tool {
       Configuration conf, String appId) throws ClientHandlerException,
       UniformInterfaceException, JSONException {
     Client webServiceClient = Client.create();
-    String webAppAddress = WebAppUtils.getRMWebAppURLWithScheme(conf);
+    String webAppAddress = WebAppUtils.getHttpSchemePrefix(conf) +
+        WebAppUtils.getWebAppBindURL(conf, YarnConfiguration.RM_BIND_HOST,
+        WebAppUtils.getRMWebAppURLWithoutScheme(conf));
     WebResource webResource = webServiceClient.resource(webAppAddress);
 
     ClientResponse response =


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[07/51] [abbrv] hadoop git commit: YARN-5206. RegistrySecurity includes id:pass in exception text if considered invalid. Contributed by Steve Loughran

Posted by vv...@apache.org.
YARN-5206. RegistrySecurity includes id:pass in exception text if considered invalid. Contributed by Steve Loughran


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

Branch: refs/heads/YARN-3926
Commit: 8c8a377cac10b086a7ff37ee366b79e6b04d2738
Parents: 723432b
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Jun 8 14:11:25 2016 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Jun 8 14:11:25 2016 +0000

----------------------------------------------------------------------
 .../apache/hadoop/registry/client/impl/zk/RegistrySecurity.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c8a377c/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 fc61460..49673fc 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
@@ -443,7 +443,7 @@ public class RegistrySecurity extends AbstractService {
    */
   public String digest(String idPasswordPair) throws IOException {
     if (StringUtils.isEmpty(idPasswordPair) || !isValid(idPasswordPair)) {
-      throw new IOException("Invalid id:password: " + idPasswordPair);
+      throw new IOException("Invalid id:password");
     }
     try {
       return DigestAuthenticationProvider.generateDigest(idPasswordPair);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[03/51] [abbrv] hadoop git commit: MAPREDUCE-6702. TestMiniMRChildTask.testTaskEnv and TestMiniMRChildTask.testTaskOldEnv are failing (ajisakaa via rkanter)

Posted by vv...@apache.org.
MAPREDUCE-6702. TestMiniMRChildTask.testTaskEnv and TestMiniMRChildTask.testTaskOldEnv are failing (ajisakaa via rkanter)


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

Branch: refs/heads/YARN-3926
Commit: 733f3f18d5cf38cbae35146fbef8e16e35fdf5e1
Parents: 620325e
Author: Robert Kanter <rk...@apache.org>
Authored: Tue Jun 7 15:46:06 2016 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Tue Jun 7 15:46:06 2016 -0700

----------------------------------------------------------------------
 .../src/site/markdown/SingleCluster.md.vm       |  12 +-
 .../java/org/apache/hadoop/mapred/JobConf.java  |   6 -
 .../hadoop/mapred/TestMiniMRChildTask.java      | 233 +++----------------
 3 files changed, 41 insertions(+), 210 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/733f3f18/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm b/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm
index 573ca32..4825e00 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/SingleCluster.md.vm
@@ -181,13 +181,23 @@ You can run a MapReduce job on YARN in a pseudo-distributed mode by setting a fe
 
 The following instructions assume that 1. ~ 4. steps of [the above instructions](#Execution) are already executed.
 
-1.  Configure parameters as follows:`etc/hadoop/mapred-site.xml`:
+1.  Configure parameters as follows:
+
+    `etc/hadoop/mapred-site.xml`:
 
         <configuration>
             <property>
                 <name>mapreduce.framework.name</name>
                 <value>yarn</value>
             </property>
+            <property>
+                <name>mapreduce.admin.user.env</name>
+                <value>HADOOP_MAPRED_HOME=$HADOOP_COMMON_HOME</value>
+            </property>
+            <property>
+                <name>yarn.app.mapreduce.am.env</name>
+                <value>HADOOP_MAPRED_HOME=$HADOOP_COMMON_HOME</value>
+            </property>
         </configuration>
 
     `etc/hadoop/yarn-site.xml`:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/733f3f18/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
index 2cfce1f..f2b0aae 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
@@ -294,8 +294,6 @@ public class JobConf extends Configuration {
    * Example:
    * <ul>
    *   <li> A=foo - This will set the env variable A to foo. </li>
-   *   <li> B=$X:c This is inherit tasktracker's X env variable on Linux. </li>
-   *   <li> B=%X%;c This is inherit tasktracker's X env variable on Windows. </li>
    * </ul>
    * 
    * @deprecated Use {@link #MAPRED_MAP_TASK_ENV} or 
@@ -314,8 +312,6 @@ public class JobConf extends Configuration {
    * Example:
    * <ul>
    *   <li> A=foo - This will set the env variable A to foo. </li>
-   *   <li> B=$X:c This is inherit tasktracker's X env variable on Linux. </li>
-   *   <li> B=%X%;c This is inherit tasktracker's X env variable on Windows. </li>
    * </ul>
    */
   public static final String MAPRED_MAP_TASK_ENV = JobContext.MAP_ENV;
@@ -330,8 +326,6 @@ public class JobConf extends Configuration {
    * Example:
    * <ul>
    *   <li> A=foo - This will set the env variable A to foo. </li>
-   *   <li> B=$X:c This is inherit tasktracker's X env variable on Linux. </li>
-   *   <li> B=%X%;c This is inherit tasktracker's X env variable on Windows. </li>
    * </ul>
    */
   public static final String MAPRED_REDUCE_TASK_ENV = JobContext.REDUCE_ENV;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/733f3f18/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
index cbeeccf..f690118 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
@@ -25,9 +25,7 @@ import static org.junit.Assert.fail;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.Iterator;
-import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -41,11 +39,9 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
-import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.util.Shell;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -111,29 +107,6 @@ public class TestMiniMRChildTask {
         }
       }
   }
-  
-  /**
-   * Map class which checks if hadoop lib location 
-   * is in the execution path
-   */
-  public static class ExecutionEnvCheckMapClass extends MapReduceBase
-      implements Mapper<LongWritable, Text, Text, IntWritable> {
-      public void map (LongWritable key, Text value, 
-          OutputCollector<Text, IntWritable> output, 
-          Reporter reporter) throws IOException {
-      }
-      public void configure(JobConf job) {
-        String executionEnvPathVariable = System.getenv(Shell.WINDOWS ? "PATH"
-            : "LD_LIBRARY_PATH");
-        String hadoopHome = System.getenv("HADOOP_COMMON_HOME");
-        if (hadoopHome == null) {
-          hadoopHome = "";
-        }
-        String hadoopLibLocation = hadoopHome 
-            + (Shell.WINDOWS ? "\\bin" : "/lib/native");
-        assertTrue(executionEnvPathVariable.contains(hadoopLibLocation));
-      }
-  }
 
   // configure a job
   private void configure(JobConf conf, Path inDir, Path outDir, String input,
@@ -167,57 +140,31 @@ public class TestMiniMRChildTask {
     conf.set("test.build.data", TEST_ROOT_DIR);
   }
 
-  /**
-   * Launch tests 
-   * @param conf Configuration of the mapreduce job.
-   * @param inDir input path
-   * @param outDir output path
-   * @param input Input text
-   * @throws IOException
-   */
-  public void launchTest(JobConf conf,
-                         Path inDir,
-                         Path outDir,
-                         String input)
-  throws IOException, InterruptedException, ClassNotFoundException {
-
-    FileSystem outFs = outDir.getFileSystem(conf);
-    
-    // Launch job with default option for temp dir. 
-    // i.e. temp dir is ./tmp 
-    Job job = Job.getInstance(conf);
-    job.addFileToClassPath(APP_JAR);
-    job.setJarByClass(TestMiniMRChildTask.class);
-    job.setMaxMapAttempts(1); // speed up failures
-    job.waitForCompletion(true);
-    boolean succeeded = job.waitForCompletion(true);
-    assertTrue(succeeded);
-    outFs.delete(outDir, true);
-  }
-
   private static void checkEnv(String envName, String expValue, String mode) {
-    String envValue = System.getenv(envName).trim();
+    String envValue = System.getenv(envName);
     if ("append".equals(mode)) {
       if (envValue == null || !envValue.contains(File.pathSeparator)) {
         throw new RuntimeException("Missing env variable");
       } else {
-        String parts[] = envValue.split(File.pathSeparator);
+        String[] parts = envValue.trim().split(File.pathSeparator);
         // check if the value is appended
         if (!parts[parts.length - 1].equals(expValue)) {
           throw new RuntimeException("Wrong env variable in append mode");
         }
       }
     } else {
-      if (envValue == null || !envValue.equals(expValue)) {
+      if (envValue == null || !envValue.trim().equals(expValue)) {
         throw new RuntimeException("Wrong env variable in noappend mode");
       }
     }
   }
 
   // Mappers that simply checks if the desired user env are present or not
-  static class EnvCheckMapper extends MapReduceBase implements
+  private static class EnvCheckMapper extends MapReduceBase implements
       Mapper<WritableComparable, Writable, WritableComparable, Writable> {
-    
+
+    @Override
+    @SuppressWarnings("deprecation")
     public void configure(JobConf job) {
       boolean oldConfigs = job.getBoolean(OLD_CONFIGS, false);
       if (oldConfigs) {
@@ -236,38 +183,12 @@ public class TestMiniMRChildTask {
                      mapJavaOpts, MAP_OPTS_VAL);
       }
 
-      String path = job.get("path");
-      
-      // check if the pwd is there in LD_LIBRARY_PATH
-      String pwd = System.getenv("PWD");
-      
-      assertTrue("LD doesnt contain pwd", 
-                 System.getenv("LD_LIBRARY_PATH").contains(pwd));
-      
-      // check if X=$X:/abc works for LD_LIBRARY_PATH
-      checkEnv("LD_LIBRARY_PATH", "/tmp", "append");
       // check if X=y works for an already existing parameter
       checkEnv("LANG", "en_us_8859_1", "noappend");
       // check if X=/tmp for a new env variable
       checkEnv("MY_PATH", "/tmp", "noappend");
       // check if X=$X:/tmp works for a new env var and results into :/tmp
       checkEnv("NEW_PATH", File.pathSeparator + "/tmp", "noappend");
-      // check if X=$(tt's X var):/tmp for an old env variable inherited from 
-      // the tt
-      if (Shell.WINDOWS) {
-        // On Windows, PATH is replaced one more time as part of default config
-        // of "mapreduce.admin.user.env", i.e. on Windows,
-        // "mapreduce.admin.user.env" is set to
-        // "PATH=%PATH%;%HADOOP_COMMON_HOME%\\bin"
-        String hadoopHome = System.getenv("HADOOP_COMMON_HOME");
-        if (hadoopHome == null) {
-          hadoopHome = "";
-        }
-        String hadoopLibLocation = hadoopHome + "\\bin";
-        path += File.pathSeparator + hadoopLibLocation;
-        path += File.pathSeparator + path;
-      }
-      checkEnv("PATH",  path + File.pathSeparator + "/tmp", "noappend");
 
       String jobLocalDir = job.get(MRJobConfig.JOB_LOCAL_DIR);
       assertNotNull(MRJobConfig.JOB_LOCAL_DIR + " is null",
@@ -281,10 +202,12 @@ public class TestMiniMRChildTask {
     }
   }
 
-  static class EnvCheckReducer extends MapReduceBase 
-  implements Reducer<WritableComparable, Writable, WritableComparable, Writable> {
-    
+  private static class EnvCheckReducer extends MapReduceBase
+      implements Reducer<WritableComparable, Writable,
+                         WritableComparable, Writable> {
+
     @Override
+    @SuppressWarnings("deprecation")
     public void configure(JobConf job) {
       boolean oldConfigs = job.getBoolean(OLD_CONFIGS, false);
       if (oldConfigs) {
@@ -303,39 +226,12 @@ public class TestMiniMRChildTask {
                      reduceJavaOpts, REDUCE_OPTS_VAL);
       }
 
-      String path = job.get("path");
-      
-      // check if the pwd is there in LD_LIBRARY_PATH
-      String pwd = System.getenv("PWD");
-      
-      assertTrue("LD doesnt contain pwd", 
-                 System.getenv("LD_LIBRARY_PATH").contains(pwd));
-      
-      // check if X=$X:/abc works for LD_LIBRARY_PATH
-      checkEnv("LD_LIBRARY_PATH", "/tmp", "append");
       // check if X=y works for an already existing parameter
       checkEnv("LANG", "en_us_8859_1", "noappend");
       // check if X=/tmp for a new env variable
       checkEnv("MY_PATH", "/tmp", "noappend");
       // check if X=$X:/tmp works for a new env var and results into :/tmp
       checkEnv("NEW_PATH", File.pathSeparator + "/tmp", "noappend");
-      // check if X=$(tt's X var):/tmp for an old env variable inherited from 
-      // the tt
-      if (Shell.WINDOWS) {
-        // On Windows, PATH is replaced one more time as part of default config
-        // of "mapreduce.admin.user.env", i.e. on Windows,
-        // "mapreduce.admin.user.env"
-        // is set to "PATH=%PATH%;%HADOOP_COMMON_HOME%\\bin"
-        String hadoopHome = System.getenv("HADOOP_COMMON_HOME");
-        if (hadoopHome == null) {
-          hadoopHome = "";
-        }
-        String hadoopLibLocation = hadoopHome + "\\bin";
-        path += File.pathSeparator + hadoopLibLocation;
-        path += File.pathSeparator + path;
-      }
-      checkEnv("PATH",  path + File.pathSeparator + "/tmp", "noappend");
-
     }
 
     @Override
@@ -393,72 +289,9 @@ public class TestMiniMRChildTask {
   }
  
   /**
-   * To test OS dependent setting of default execution path for a MapRed task.
-   * Mainly that we can use MRJobConfig.DEFAULT_MAPRED_ADMIN_USER_ENV to set -
-   * for WINDOWS: %HADOOP_COMMON_HOME%\bin is expected to be included in PATH -
-   * for Linux: $HADOOP_COMMON_HOME/lib/native is expected to be included in
-   * LD_LIBRARY_PATH
-   */
-  @Test
-  public void testMapRedExecutionEnv() {
-    // for windows, test if the env variable can be set
-    // this may be removed as part of MAPREDUCE-6588
-    if (Shell.WINDOWS) {
-      try {
-        // Application environment
-        Map<String, String> environment = new HashMap<String, String>();
-        String setupHadoopHomeCommand =
-          "HADOOP_COMMON_HOME=C:\\fake\\PATH\\to\\hadoop\\common\\home";
-        MRApps.setEnvFromInputString(environment, setupHadoopHomeCommand, conf);
-
-        // Add the env variables passed by the admin
-        MRApps.setEnvFromInputString(environment, conf.get(
-            MRJobConfig.MAPRED_ADMIN_USER_ENV,
-            MRJobConfig.DEFAULT_MAPRED_ADMIN_USER_ENV), conf);
-
-        String executionPaths = environment.get("PATH");
-        String toFind =
-            "C:\\fake\\PATH\\to\\hadoop\\common\\home\\bin";
-
-        // Ensure execution PATH/LD_LIBRARY_PATH set up pointing to hadoop lib
-        assertTrue("execution path does not include the hadoop lib location "
-            + toFind, executionPaths.contains(toFind));
-      } catch (Exception e) {
-        e.printStackTrace();
-        fail("Exception in testing execution environment for MapReduce task");
-        tearDown();
-      }
-    }
-
-    // now launch a mapreduce job to ensure that the child 
-    // also gets the configured setting for hadoop lib
-    try {
-
-      JobConf conf = new JobConf(mr.getConfig());
-      // initialize input, output directories
-      Path rootDir = new Path(System.getProperty("test.build.data",
-          "build/test/data"));
-      Path inDir = new Path(rootDir, "input");
-      Path outDir = new Path(rootDir, "output");
-      String input = "The input";
-
-      // set config to use the ExecutionEnvCheckMapClass map class
-      configure(conf, inDir, outDir, input, 
-          ExecutionEnvCheckMapClass.class, IdentityReducer.class);
-      launchTest(conf, inDir, outDir, input);
-
-    } catch(Exception e) {
-      e.printStackTrace();
-      fail("Exception in testing propagation of env setting to child task");
-      tearDown();
-    }
-  }
-  
-  /**
    * Test to test if the user set env variables reflect in the child
    * processes. Mainly
    *   - x=y (x can be a already existing env variable or a new variable)
-   *   - x=$x:y (replace $x with the current value of x)
    */
   @Test
   public void testTaskEnv(){
@@ -482,10 +315,9 @@ public class TestMiniMRChildTask {
    * Test to test if the user set *old* env variables reflect in the child
    * processes. Mainly
    *   - x=y (x can be a already existing env variable or a new variable)
-   *   - x=$x:y (replace $x with the current value of x)
    */
   @Test
-  public void testTaskOldEnv(){
+  public void testTaskOldEnv() {
     try {
       JobConf conf = new JobConf(mr.getConfig());
       String baseDir = System.getProperty("test.build.data", "build/test/data");
@@ -495,23 +327,23 @@ public class TestMiniMRChildTask {
       FileSystem outFs = outDir.getFileSystem(conf);
       runTestTaskEnv(conf, inDir, outDir, true);
       outFs.delete(outDir, true);
-    } catch(Exception e) {
+    } catch (Exception e) {
       e.printStackTrace();
       fail("Exception in testing child env");
       tearDown();
     }
   }
-  
-  void runTestTaskEnv(JobConf conf, Path inDir, Path outDir, boolean oldConfigs) 
-  throws IOException, InterruptedException, ClassNotFoundException {
+
+  @SuppressWarnings("deprecation")
+  private void runTestTaskEnv(JobConf config, Path inDir, Path outDir,
+                              boolean oldConfigs)
+      throws IOException, InterruptedException, ClassNotFoundException {
     String input = "The input";
-    configure(conf, inDir, outDir, input, 
+    configure(config, inDir, outDir, input,
               EnvCheckMapper.class, EnvCheckReducer.class);
     // test 
     //  - new SET of new var (MY_PATH)
     //  - set of old var (LANG)
-    //  - append to an old var from modified env (LD_LIBRARY_PATH)
-    //  - append to an old var from tt's env (PATH)
     //  - append to a new var (NEW_PATH)
     String mapTaskEnvKey = JobConf.MAPRED_MAP_TASK_ENV;
     String reduceTaskEnvKey = JobConf.MAPRED_MAP_TASK_ENV;
@@ -519,29 +351,24 @@ public class TestMiniMRChildTask {
     String reduceTaskJavaOptsKey = JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS;
     String mapTaskJavaOpts = MAP_OPTS_VAL;
     String reduceTaskJavaOpts = REDUCE_OPTS_VAL;
-    conf.setBoolean(OLD_CONFIGS, oldConfigs);
+    config.setBoolean(OLD_CONFIGS, oldConfigs);
     if (oldConfigs) {
       mapTaskEnvKey = reduceTaskEnvKey = JobConf.MAPRED_TASK_ENV;
       mapTaskJavaOptsKey = reduceTaskJavaOptsKey = JobConf.MAPRED_TASK_JAVA_OPTS;
       mapTaskJavaOpts = reduceTaskJavaOpts = TASK_OPTS_VAL;
     }
-    conf.set(
+    config.set(
         mapTaskEnvKey,
-        Shell.WINDOWS ? "MY_PATH=/tmp,LANG=en_us_8859_1,LD_LIBRARY_PATH=%LD_LIBRARY_PATH%;/tmp,"
-            + "PATH=%PATH%;/tmp,NEW_PATH=%NEW_PATH%;/tmp"
-            : "MY_PATH=/tmp,LANG=en_us_8859_1,LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/tmp,"
-                + "PATH=$PATH:/tmp,NEW_PATH=$NEW_PATH:/tmp");
-    conf.set(
+        Shell.WINDOWS ? "MY_PATH=/tmp,LANG=en_us_8859_1,NEW_PATH=%MY_PATH%;/tmp"
+            : "MY_PATH=/tmp,LANG=en_us_8859_1,NEW_PATH=$NEW_PATH:/tmp");
+    config.set(
         reduceTaskEnvKey,
-        Shell.WINDOWS ? "MY_PATH=/tmp,LANG=en_us_8859_1,LD_LIBRARY_PATH=%LD_LIBRARY_PATH%;/tmp,"
-            + "PATH=%PATH%;/tmp,NEW_PATH=%NEW_PATH%;/tmp"
-            : "MY_PATH=/tmp,LANG=en_us_8859_1,LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/tmp,"
-                + "PATH=$PATH:/tmp,NEW_PATH=$NEW_PATH:/tmp");
-    conf.set("path", System.getenv("PATH"));
-    conf.set(mapTaskJavaOptsKey, mapTaskJavaOpts);
-    conf.set(reduceTaskJavaOptsKey, reduceTaskJavaOpts);
+        Shell.WINDOWS ? "MY_PATH=/tmp,LANG=en_us_8859_1,NEW_PATH=%MY_PATH%;/tmp"
+            : "MY_PATH=/tmp,LANG=en_us_8859_1,NEW_PATH=$NEW_PATH:/tmp");
+    config.set(mapTaskJavaOptsKey, mapTaskJavaOpts);
+    config.set(reduceTaskJavaOptsKey, reduceTaskJavaOpts);
 
-    Job job = Job.getInstance(conf);
+    Job job = Job.getInstance(config);
     job.addFileToClassPath(APP_JAR);
     job.setJarByClass(TestMiniMRChildTask.class);
     job.setMaxMapAttempts(1); // speed up failures


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[36/51] [abbrv] hadoop git commit: HADOOP-13249. RetryInvocationHandler need wrap InterruptedException in IOException when call Thread.sleep. Contributed by Zhihai Xu.

Posted by vv...@apache.org.
HADOOP-13249. RetryInvocationHandler need wrap InterruptedException in IOException when call Thread.sleep. Contributed by Zhihai Xu.


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

Branch: refs/heads/YARN-3926
Commit: 0bbb4ddd793063c87927035969884a34f60f2076
Parents: 03fc6b1
Author: Jing Zhao <ji...@apache.org>
Authored: Fri Jun 10 10:38:13 2016 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Fri Jun 10 10:38:13 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/io/retry/RetryInvocationHandler.java  | 12 +++++++++++-
 .../java/org/apache/hadoop/io/retry/TestRetryProxy.java |  7 +++++--
 2 files changed, 16 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bbb4ddd/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
index f2b2c99..5198c0d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.ipc.Client.ConnectionId;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
@@ -297,7 +298,16 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
     log(method, isFailover, counters.failovers, retryInfo.delay, ex);
 
     if (retryInfo.delay > 0) {
-      Thread.sleep(retryInfo.delay);
+      try {
+        Thread.sleep(retryInfo.delay);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        LOG.warn("Interrupted while waiting to retry", e);
+        InterruptedIOException intIOE = new InterruptedIOException(
+            "Retry interrupted");
+        intIOE.initCause(e);
+        throw intIOE;
+      }
     }
 
     if (isFailover) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bbb4ddd/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java
index 41c1be4..649af89 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java
@@ -31,6 +31,7 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.util.Collections;
 import java.util.Map;
@@ -320,7 +321,9 @@ public class TestRetryProxy {
     futureThread.get().interrupt();
     Throwable e = future.get(1, TimeUnit.SECONDS); // should return immediately 
     assertNotNull(e);
-    assertEquals(InterruptedException.class, e.getClass());
-    assertEquals("sleep interrupted", e.getMessage());
+    assertEquals(InterruptedIOException.class, e.getClass());
+    assertEquals("Retry interrupted", e.getMessage());
+    assertEquals(InterruptedException.class, e.getCause().getClass());
+    assertEquals("sleep interrupted", e.getCause().getMessage());
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[27/51] [abbrv] hadoop git commit: YARN-3426. Add jdiff support to YARN. (vinodkv via wangda)

Posted by vv...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/03fc6b1b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_2.7.2.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_2.7.2.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_2.7.2.xml
new file mode 100644
index 0000000..1a1d88b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_2.7.2.xml
@@ -0,0 +1,1801 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Thu May 12 17:47:55 PDT 2016 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="hadoop-yarn-server-common 2.7.2"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.ExcludePrivateAnnotationsJDiffDoclet -docletpath /Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/target/hadoop-annotations.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/target/jdiff.jar -verbose -classpath /Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/target/classes:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-common-project/hadoop-common/target/hadoop-common-2.7.2.jar:/Users/vinodkv/.m2/repository/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/Users/vinodkv/.m2/repository/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/Users/vinodkv/.m2/repository/xmlenc/xmlenc/0.52/xmlenc-0.52.jar:/Users/vi
 nodkv/.m2/repository/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1.jar:/Users/vinodkv/.m2/repository/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/Users/vinodkv/.m2/repository/commons-io/commons-io/2.4/commons-io-2.4.jar:/Users/vinodkv/.m2/repository/commons-net/commons-net/3.1/commons-net-3.1.jar:/Users/vinodkv/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/Users/vinodkv/.m2/repository/javax/servlet/servlet-api/2.5/servlet-api-2.5.jar:/Users/vinodkv/.m2/repository/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/Users/vinodkv/.m2/repository/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/Users/vinodkv/.m2/repository/javax/servlet/jsp/jsp-api/2.1/jsp-api-2.1.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/jersey-core/1.9/jersey-core-1.9.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/jersey-json/1.9/jersey-json-1.9.jar:/Users/vinodkv/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.ja
 r:/Users/vinodkv/.m2/repository/com/sun/xml/bind/jaxb-impl/2.2.3-1/jaxb-impl-2.2.3-1.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/jersey-server/1.9/jersey-server-1.9.jar:/Users/vinodkv/.m2/repository/asm/asm/3.2/asm-3.2.jar:/Users/vinodkv/.m2/repository/log4j/log4j/1.2.17/log4j-1.2.17.jar:/Users/vinodkv/.m2/repository/net/java/dev/jets3t/jets3t/0.9.0/jets3t-0.9.0.jar:/Users/vinodkv/.m2/repository/org/apache/httpcomponents/httpclient/4.2.5/httpclient-4.2.5.jar:/Users/vinodkv/.m2/repository/org/apache/httpcomponents/httpcore/4.2.5/httpcore-4.2.5.jar:/Users/vinodkv/.m2/repository/com/jamesmurty/utils/java-xmlbuilder/0.4/java-xmlbuilder-0.4.jar:/Users/vinodkv/.m2/repository/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/Users/vinodkv/.m2/repository/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/Users/vinodkv/.m2/repository/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/Users/vinodkv/.m2/repository/commons-beanutils/commons-beanut
 ils/1.7.0/commons-beanutils-1.7.0.jar:/Users/vinodkv/.m2/repository/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar:/Users/vinodkv/.m2/repository/org/slf4j/slf4j-api/1.7.10/slf4j-api-1.7.10.jar:/Users/vinodkv/.m2/repository/org/slf4j/slf4j-log4j12/1.7.10/slf4j-log4j12-1.7.10.jar:/Users/vinodkv/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/vinodkv/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/vinodkv/.m2/repository/org/apache/avro/avro/1.7.4/avro-1.7.4.jar:/Users/vinodkv/.m2/repository/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/Users/vinodkv/.m2/repository/org/xerial/snappy/snappy-java/1.0.4.1/snappy-java-1.0.4.1.jar:/Users/vinodkv/.m2/repository/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-common-project/hadoop-auth/target/hadoop-auth-2.7.2.jar:/Users/vinodkv/.m2/re
 pository/org/apache/directory/server/apacheds-kerberos-codec/2.0.0-M15/apacheds-kerberos-codec-2.0.0-M15.jar:/Users/vinodkv/.m2/repository/org/apache/directory/server/apacheds-i18n/2.0.0-M15/apacheds-i18n-2.0.0-M15.jar:/Users/vinodkv/.m2/repository/org/apache/directory/api/api-asn1-api/1.0.0-M20/api-asn1-api-1.0.0-M20.jar:/Users/vinodkv/.m2/repository/org/apache/directory/api/api-util/1.0.0-M20/api-util-1.0.0-M20.jar:/Users/vinodkv/.m2/repository/org/apache/curator/curator-framework/2.7.1/curator-framework-2.7.1.jar:/Users/vinodkv/.m2/repository/com/jcraft/jsch/0.1.42/jsch-0.1.42.jar:/Users/vinodkv/.m2/repository/org/apache/curator/curator-client/2.7.1/curator-client-2.7.1.jar:/Users/vinodkv/.m2/repository/org/apache/curator/curator-recipes/2.7.1/curator-recipes-2.7.1.jar:/Users/vinodkv/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/vinodkv/.m2/repository/org/apache/htrace/htrace-core/3.1.0-incubating/htrace-core-3.1.0-incubating.jar:/Users/vinodkv/.m2/
 repository/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/Users/vinodkv/.m2/repository/org/tukaani/xz/1.0/xz-1.0.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/hadoop-yarn-api-2.7.2.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/hadoop-yarn-common-2.7.2.jar:/Users/vinodkv/.m2/repository/javax/xml/bind/jaxb-api/2.2.2/jaxb-api-2.2.2.jar:/Users/vinodkv/.m2/repository/javax/xml/stream/stax-api/1.0-2/stax-api-1.0-2.jar:/Users/vinodkv/.m2/repository/javax/activation/activation/1.1/activation-1.1.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/jersey-client/1.9/jersey-client-1.9.jar:/Users/vinodkv/.m2/repository/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/Users/vinodkv/.m2/repository/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/Users/vinodkv/.m2/repository/com/google/inject/exten
 sions/guice-servlet/3.0/guice-servlet-3.0.jar:/Users/vinodkv/.m2/repository/com/google/inject/guice/3.0/guice-3.0.jar:/Users/vinodkv/.m2/repository/javax/inject/javax.inject/1/javax.inject-1.jar:/Users/vinodkv/.m2/repository/aopalliance/aopalliance/1.0/aopalliance-1.0.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/contribs/jersey-guice/1.9/jersey-guice-1.9.jar:/Users/vinodkv/.m2/repository/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/Users/vinodkv/.m2/repository/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-common-project/hadoop-annotations/target/hadoop-annotations-2.7.2.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_45.jdk/Contents/Home/lib/tools.jar:/Users/vinodkv/.m2/repository/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar:/Users/vinodkv/.m2/repository/org/apache/zookeeper/zookeeper/3.4.6/zookeeper-3.4.6.jar:/Users/vinodkv/.m2/repository/io/netty/netty/3.6.2.Final/netty-3
 .6.2.Final.jar:/Users/vinodkv/.m2/repository/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar -sourcepath /Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java -apidir /Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/target/site/jdiff/xml -apiname hadoop-yarn-server-common 2.7.2 -->
+<package name="org.apache.hadoop.yarn.server">
+  <!-- start class org.apache.hadoop.yarn.server.RMNMSecurityInfoClass -->
+  <class name="RMNMSecurityInfoClass" extends="org.apache.hadoop.security.SecurityInfo"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="RMNMSecurityInfoClass"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getKerberosInfo" return="org.apache.hadoop.security.KerberosInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getTokenInfo" return="org.apache.hadoop.security.token.TokenInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.RMNMSecurityInfoClass -->
+</package>
+<package name="org.apache.hadoop.yarn.server.api">
+  <!-- start interface org.apache.hadoop.yarn.server.api.ResourceManagerConstants -->
+  <interface name="ResourceManagerConstants"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <field name="RM_INVALID_IDENTIFIER" type="long"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[This states the invalid identifier of Resource Manager. This is used as a
+ default value for initializing RM identifier. Currently, RM is using time
+ stamp as RM identifier.]]>
+      </doc>
+    </field>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.server.api.ResourceManagerConstants -->
+  <!-- start interface org.apache.hadoop.yarn.server.api.ResourceTracker -->
+  <interface name="ResourceTracker"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="registerNodeManager" return="org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="nodeHeartbeat" return="org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.server.api.ResourceTracker -->
+  <!-- start interface org.apache.hadoop.yarn.server.api.ResourceTrackerPB -->
+  <interface name="ResourceTrackerPB"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.proto.ResourceTracker.ResourceTrackerService.BlockingInterface"/>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.server.api.ResourceTrackerPB -->
+  <!-- start interface org.apache.hadoop.yarn.server.api.SCMUploaderProtocolPB -->
+  <interface name="SCMUploaderProtocolPB"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.proto.SCMUploaderProtocol.SCMUploaderProtocolService.BlockingInterface"/>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.server.api.SCMUploaderProtocolPB -->
+  <!-- start class org.apache.hadoop.yarn.server.api.ServerRMProxy -->
+  <class name="ServerRMProxy" extends="org.apache.hadoop.yarn.client.RMProxy"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="createRMProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="configuration" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Create a proxy to the ResourceManager for the specified protocol.
+ @param configuration Configuration with all the required information.
+ @param protocol Server protocol for which proxy is being requested.
+ @param <T> Type of proxy.
+ @return Proxy to the ResourceManager for the specified server protocol.
+ @throws IOException]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.ServerRMProxy -->
+</package>
+<package name="org.apache.hadoop.yarn.server.api.impl.pb.client">
+  <!-- start class org.apache.hadoop.yarn.server.api.impl.pb.client.ResourceTrackerPBClientImpl -->
+  <class name="ResourceTrackerPBClientImpl" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.server.api.ResourceTracker"/>
+    <implements name="java.io.Closeable"/>
+    <constructor name="ResourceTrackerPBClientImpl" type="long, java.net.InetSocketAddress, org.apache.hadoop.conf.Configuration"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </constructor>
+    <method name="close"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="registerNodeManager" return="org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="nodeHeartbeat" return="org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.impl.pb.client.ResourceTrackerPBClientImpl -->
+  <!-- start class org.apache.hadoop.yarn.server.api.impl.pb.client.SCMUploaderProtocolPBClientImpl -->
+  <class name="SCMUploaderProtocolPBClientImpl" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.server.api.SCMUploaderProtocol"/>
+    <implements name="java.io.Closeable"/>
+    <constructor name="SCMUploaderProtocolPBClientImpl" type="long, java.net.InetSocketAddress, org.apache.hadoop.conf.Configuration"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </constructor>
+    <method name="close"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="notify" return="org.apache.hadoop.yarn.server.api.protocolrecords.SCMUploaderNotifyResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.server.api.protocolrecords.SCMUploaderNotifyRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="canUpload" return="org.apache.hadoop.yarn.server.api.protocolrecords.SCMUploaderCanUploadResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.server.api.protocolrecords.SCMUploaderCanUploadRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.impl.pb.client.SCMUploaderProtocolPBClientImpl -->
+</package>
+<package name="org.apache.hadoop.yarn.server.api.impl.pb.service">
+  <!-- start class org.apache.hadoop.yarn.server.api.impl.pb.service.ResourceTrackerPBServiceImpl -->
+  <class name="ResourceTrackerPBServiceImpl" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.server.api.ResourceTrackerPB"/>
+    <constructor name="ResourceTrackerPBServiceImpl" type="org.apache.hadoop.yarn.server.api.ResourceTracker"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="registerNodeManager" return="org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.RegisterNodeManagerResponseProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="controller" type="com.google.protobuf.RpcController"/>
+      <param name="proto" type="org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.RegisterNodeManagerRequestProto"/>
+      <exception name="ServiceException" type="com.google.protobuf.ServiceException"/>
+    </method>
+    <method name="nodeHeartbeat" return="org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="controller" type="com.google.protobuf.RpcController"/>
+      <param name="proto" type="org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProto"/>
+      <exception name="ServiceException" type="com.google.protobuf.ServiceException"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.impl.pb.service.ResourceTrackerPBServiceImpl -->
+  <!-- start class org.apache.hadoop.yarn.server.api.impl.pb.service.SCMUploaderProtocolPBServiceImpl -->
+  <class name="SCMUploaderProtocolPBServiceImpl" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.server.api.SCMUploaderProtocolPB"/>
+    <constructor name="SCMUploaderProtocolPBServiceImpl" type="org.apache.hadoop.yarn.server.api.SCMUploaderProtocol"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="notify" return="org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.SCMUploaderNotifyResponseProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="controller" type="com.google.protobuf.RpcController"/>
+      <param name="proto" type="org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.SCMUploaderNotifyRequestProto"/>
+      <exception name="ServiceException" type="com.google.protobuf.ServiceException"/>
+    </method>
+    <method name="canUpload" return="org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.SCMUploaderCanUploadResponseProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="controller" type="com.google.protobuf.RpcController"/>
+      <param name="proto" type="org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.SCMUploaderCanUploadRequestProto"/>
+      <exception name="ServiceException" type="com.google.protobuf.ServiceException"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.impl.pb.service.SCMUploaderProtocolPBServiceImpl -->
+</package>
+<package name="org.apache.hadoop.yarn.server.api.records">
+  <!-- start interface org.apache.hadoop.yarn.server.api.records.MasterKey -->
+  <interface name="MasterKey"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getKeyId" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setKeyId"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="keyId" type="int"/>
+    </method>
+    <method name="getBytes" return="java.nio.ByteBuffer"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setBytes"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="bytes" type="java.nio.ByteBuffer"/>
+    </method>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.server.api.records.MasterKey -->
+  <!-- start class org.apache.hadoop.yarn.server.api.records.NodeAction -->
+  <class name="NodeAction" extends="java.lang.Enum"
+    abstract="false"
+    static="false" final="true" visibility="public"
+    deprecated="not deprecated">
+    <method name="values" return="org.apache.hadoop.yarn.server.api.records.NodeAction[]"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="valueOf" return="org.apache.hadoop.yarn.server.api.records.NodeAction"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+    </method>
+    <doc>
+    <![CDATA[The NodeManager is instructed to perform the given action.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.records.NodeAction -->
+  <!-- start class org.apache.hadoop.yarn.server.api.records.NodeHealthStatus -->
+  <class name="NodeHealthStatus" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NodeHealthStatus"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getIsNodeHealthy" return="boolean"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Is the node healthy?
+ @return <code>true</code> if the node is healthy, else <code>false</code>]]>
+      </doc>
+    </method>
+    <method name="getHealthReport" return="java.lang.String"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <em>diagnostic health report</em> of the node.
+ @return <em>diagnostic health report</em> of the node]]>
+      </doc>
+    </method>
+    <method name="getLastHealthReportTime" return="long"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <em>last timestamp</em> at which the health report was received.
+ @return <em>last timestamp</em> at which the health report was received]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[{@code NodeHealthStatus} is a summary of the health status of the node.
+ <p>
+ It includes information such as:
+ <ul>
+   <li>
+     An indicator of whether the node is healthy, as determined by the
+     health-check script.
+   </li>
+   <li>The previous time at which the health status was reported.</li>
+   <li>A diagnostic report on the health status.</li>
+ </ul>
+
+ @see NodeReport
+ @see ApplicationClientProtocol#getClusterNodes(org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest)]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.records.NodeHealthStatus -->
+  <!-- start class org.apache.hadoop.yarn.server.api.records.NodeStatus -->
+  <class name="NodeStatus" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NodeStatus"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="newInstance" return="org.apache.hadoop.yarn.server.api.records.NodeStatus"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <param name="responseId" type="int"/>
+      <param name="containerStatuses" type="java.util.List"/>
+      <param name="keepAliveApplications" type="java.util.List"/>
+      <param name="nodeHealthStatus" type="org.apache.hadoop.yarn.server.api.records.NodeHealthStatus"/>
+    </method>
+    <method name="getNodeId" return="org.apache.hadoop.yarn.api.records.NodeId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getResponseId" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getContainersStatuses" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setContainersStatuses"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containersStatuses" type="java.util.List"/>
+    </method>
+    <method name="getKeepAliveApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setKeepAliveApplications"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appIds" type="java.util.List"/>
+    </method>
+    <method name="getNodeHealthStatus" return="org.apache.hadoop.yarn.server.api.records.NodeHealthStatus"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setNodeHealthStatus"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="healthStatus" type="org.apache.hadoop.yarn.server.api.records.NodeHealthStatus"/>
+    </method>
+    <method name="setNodeId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+    </method>
+    <method name="setResponseId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="responseId" type="int"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.records.NodeStatus -->
+</package>
+<package name="org.apache.hadoop.yarn.server.api.records.impl.pb">
+  <!-- start class org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl -->
+  <class name="MasterKeyPBImpl" extends="org.apache.hadoop.yarn.api.records.impl.pb.ProtoBase"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.server.api.records.MasterKey"/>
+    <constructor name="MasterKeyPBImpl"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="MasterKeyPBImpl" type="org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getKeyId" return="int"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setKeyId"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="id" type="int"/>
+    </method>
+    <method name="getBytes" return="java.nio.ByteBuffer"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setBytes"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="bytes" type="java.nio.ByteBuffer"/>
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="obj" type="java.lang.Object"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl -->
+  <!-- start class org.apache.hadoop.yarn.server.api.records.impl.pb.NodeHealthStatusPBImpl -->
+  <class name="NodeHealthStatusPBImpl" extends="org.apache.hadoop.yarn.server.api.records.NodeHealthStatus"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NodeHealthStatusPBImpl"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="NodeHealthStatusPBImpl" type="org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeHealthStatusProto"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeHealthStatusProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="other" type="java.lang.Object"/>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getIsNodeHealthy" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setIsNodeHealthy"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="isNodeHealthy" type="boolean"/>
+    </method>
+    <method name="getHealthReport" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setHealthReport"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="healthReport" type="java.lang.String"/>
+    </method>
+    <method name="getLastHealthReportTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setLastHealthReportTime"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="lastHealthReport" type="long"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.records.impl.pb.NodeHealthStatusPBImpl -->
+  <!-- start class org.apache.hadoop.yarn.server.api.records.impl.pb.NodeStatusPBImpl -->
+  <class name="NodeStatusPBImpl" extends="org.apache.hadoop.yarn.server.api.records.NodeStatus"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NodeStatusPBImpl"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="NodeStatusPBImpl" type="org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="other" type="java.lang.Object"/>
+    </method>
+    <method name="getResponseId" return="int"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setResponseId"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="responseId" type="int"/>
+    </method>
+    <method name="getNodeId" return="org.apache.hadoop.yarn.api.records.NodeId"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setNodeId"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+    </method>
+    <method name="getContainersStatuses" return="java.util.List"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setContainersStatuses"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containers" type="java.util.List"/>
+    </method>
+    <method name="getKeepAliveApplications" return="java.util.List"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setKeepAliveApplications"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appIds" type="java.util.List"/>
+    </method>
+    <method name="getNodeHealthStatus" return="org.apache.hadoop.yarn.server.api.records.NodeHealthStatus"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setNodeHealthStatus"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="healthStatus" type="org.apache.hadoop.yarn.server.api.records.NodeHealthStatus"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.records.impl.pb.NodeStatusPBImpl -->
+</package>
+<package name="org.apache.hadoop.yarn.server.metrics">
+</package>
+<package name="org.apache.hadoop.yarn.server.records">
+</package>
+<package name="org.apache.hadoop.yarn.server.records.impl.pb">
+  <!-- start class org.apache.hadoop.yarn.server.records.impl.pb.VersionPBImpl -->
+  <class name="VersionPBImpl" extends="org.apache.hadoop.yarn.server.records.Version"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="VersionPBImpl"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="VersionPBImpl" type="org.apache.hadoop.yarn.proto.YarnServerCommonProtos.VersionProto"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnServerCommonProtos.VersionProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getMajorVersion" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setMajorVersion"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="major" type="int"/>
+    </method>
+    <method name="getMinorVersion" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setMinorVersion"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="minor" type="int"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.records.impl.pb.VersionPBImpl -->
+</package>
+<package name="org.apache.hadoop.yarn.server.security.http">
+  <!-- start class org.apache.hadoop.yarn.server.security.http.RMAuthenticationFilterInitializer -->
+  <class name="RMAuthenticationFilterInitializer" extends="org.apache.hadoop.http.FilterInitializer"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="RMAuthenticationFilterInitializer"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createFilterConfig" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="initFilter"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="container" type="org.apache.hadoop.http.FilterContainer"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.security.http.RMAuthenticationFilterInitializer -->
+</package>
+<package name="org.apache.hadoop.yarn.server.sharedcache">
+</package>
+<package name="org.apache.hadoop.yarn.server.utils">
+  <!-- start class org.apache.hadoop.yarn.server.utils.BuilderUtils.ApplicationIdComparator -->
+  <class name="BuilderUtils.ApplicationIdComparator" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.util.Comparator"/>
+    <implements name="java.io.Serializable"/>
+    <constructor name="BuilderUtils.ApplicationIdComparator"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="compare" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="a1" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="a2" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.utils.BuilderUtils.ApplicationIdComparator -->
+  <!-- start class org.apache.hadoop.yarn.server.utils.BuilderUtils.ContainerIdComparator -->
+  <class name="BuilderUtils.ContainerIdComparator" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.util.Comparator"/>
+    <implements name="java.io.Serializable"/>
+    <constructor name="BuilderUtils.ContainerIdComparator"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="compare" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="c1" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <param name="c2" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.utils.BuilderUtils.ContainerIdComparator -->
+  <!-- start class org.apache.hadoop.yarn.server.utils.LeveldbIterator -->
+  <class name="LeveldbIterator" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.util.Iterator"/>
+    <implements name="java.io.Closeable"/>
+    <constructor name="LeveldbIterator" type="org.iq80.leveldb.DB"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create an iterator for the specified database]]>
+      </doc>
+    </constructor>
+    <constructor name="LeveldbIterator" type="org.iq80.leveldb.DB, org.iq80.leveldb.ReadOptions"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create an iterator for the specified database]]>
+      </doc>
+    </constructor>
+    <constructor name="LeveldbIterator" type="org.iq80.leveldb.DBIterator"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create an iterator using the specified underlying DBIterator]]>
+      </doc>
+    </constructor>
+    <method name="seek"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="key" type="byte[]"/>
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Repositions the iterator so the key of the next BlockElement
+ returned greater than or equal to the specified targetKey.]]>
+      </doc>
+    </method>
+    <method name="seekToFirst"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Repositions the iterator so is is at the beginning of the Database.]]>
+      </doc>
+    </method>
+    <method name="seekToLast"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Repositions the iterator so it is at the end of of the Database.]]>
+      </doc>
+    </method>
+    <method name="hasNext" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Returns <tt>true</tt> if the iteration has more elements.]]>
+      </doc>
+    </method>
+    <method name="next" return="java.util.Map.Entry"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Returns the next element in the iteration.]]>
+      </doc>
+    </method>
+    <method name="peekNext" return="java.util.Map.Entry"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Returns the next element in the iteration, without advancing the
+ iteration.]]>
+      </doc>
+    </method>
+    <method name="hasPrev" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[@return true if there is a previous entry in the iteration.]]>
+      </doc>
+    </method>
+    <method name="prev" return="java.util.Map.Entry"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[@return the previous element in the iteration and rewinds the iteration.]]>
+      </doc>
+    </method>
+    <method name="peekPrev" return="java.util.Map.Entry"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[@return the previous element in the iteration, without rewinding the
+ iteration.]]>
+      </doc>
+    </method>
+    <method name="remove"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Removes from the database the last element returned by the iterator.]]>
+      </doc>
+    </method>
+    <method name="close"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Closes the iterator.]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[A wrapper for a DBIterator to translate the raw RuntimeExceptions that
+ can be thrown into DBExceptions.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.utils.LeveldbIterator -->
+  <!-- start class org.apache.hadoop.yarn.server.utils.Lock -->
+  <class name="Lock"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.lang.annotation.Annotation"/>
+    <doc>
+    <![CDATA[Annotation to document locking order.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.utils.Lock -->
+  <!-- start class org.apache.hadoop.yarn.server.utils.Lock.NoLock -->
+  <class name="Lock.NoLock" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="Lock.NoLock"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.utils.Lock.NoLock -->
+  <!-- start class org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils -->
+  <class name="YarnServerBuilderUtils" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="YarnServerBuilderUtils"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="newNodeHeartbeatResponse" return="org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="responseId" type="int"/>
+      <param name="action" type="org.apache.hadoop.yarn.server.api.records.NodeAction"/>
+      <param name="containersToCleanUp" type="java.util.List"/>
+      <param name="applicationsToCleanUp" type="java.util.List"/>
+      <param name="containerTokenMasterKey" type="org.apache.hadoop.yarn.server.api.records.MasterKey"/>
+      <param name="nmTokenMasterKey" type="org.apache.hadoop.yarn.server.api.records.MasterKey"/>
+      <param name="nextHeartbeatInterval" type="long"/>
+    </method>
+    <doc>
+    <![CDATA[Server Builder utilities to construct various objects.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils -->
+</package>
+<package name="org.apache.hadoop.yarn.server.webapp">
+  <!-- start class org.apache.hadoop.yarn.server.webapp.AppAttemptBlock -->
+  <class name="AppAttemptBlock" extends="org.apache.hadoop.yarn.webapp.view.HtmlBlock"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AppAttemptBlock" type="org.apache.hadoop.yarn.api.ApplicationBaseProtocol, org.apache.hadoop.yarn.webapp.View.ViewContext"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="render"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="html" type="org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block"/>
+    </method>
+    <method name="generateOverview"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="appAttemptReport" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptReport"/>
+      <param name="containers" type="java.util.Collection"/>
+      <param name="appAttempt" type="org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo"/>
+      <param name="node" type="java.lang.String"/>
+    </method>
+    <method name="hasAMContainer" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <param name="containers" type="java.util.Collection"/>
+    </method>
+    <method name="createAttemptHeadRoomTable"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="html" type="org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block"/>
+    </method>
+    <method name="createTablesForAttemptMetrics"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="html" type="org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block"/>
+    </method>
+    <field name="appBaseProt" type="org.apache.hadoop.yarn.api.ApplicationBaseProtocol"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="appAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.AppAttemptBlock -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.AppBlock -->
+  <class name="AppBlock" extends="org.apache.hadoop.yarn.webapp.view.HtmlBlock"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AppBlock" type="org.apache.hadoop.yarn.api.ApplicationBaseProtocol, org.apache.hadoop.yarn.webapp.View.ViewContext, org.apache.hadoop.conf.Configuration"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="render"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="html" type="org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block"/>
+    </method>
+    <method name="generateApplicationTable"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="html" type="org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block"/>
+      <param name="callerUGI" type="org.apache.hadoop.security.UserGroupInformation"/>
+      <param name="attempts" type="java.util.Collection"/>
+    </method>
+    <method name="createApplicationMetricsTable"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="html" type="org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block"/>
+    </method>
+    <field name="appBaseProt" type="org.apache.hadoop.yarn.api.ApplicationBaseProtocol"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="conf" type="org.apache.hadoop.conf.Configuration"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="appID" type="org.apache.hadoop.yarn.api.records.ApplicationId"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.AppBlock -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.AppsBlock -->
+  <class name="AppsBlock" extends="org.apache.hadoop.yarn.webapp.view.HtmlBlock"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AppsBlock" type="org.apache.hadoop.yarn.api.ApplicationBaseProtocol, org.apache.hadoop.yarn.webapp.View.ViewContext"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="fetchData"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="InterruptedException" type="java.lang.InterruptedException"/>
+    </method>
+    <method name="render"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="html" type="org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block"/>
+    </method>
+    <method name="renderData"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="html" type="org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block"/>
+    </method>
+    <field name="appBaseProt" type="org.apache.hadoop.yarn.api.ApplicationBaseProtocol"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="reqAppStates" type="java.util.EnumSet"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="callerUGI" type="org.apache.hadoop.security.UserGroupInformation"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="appReports" type="java.util.Collection"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.AppsBlock -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.ContainerBlock -->
+  <class name="ContainerBlock" extends="org.apache.hadoop.yarn.webapp.view.HtmlBlock"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ContainerBlock" type="org.apache.hadoop.yarn.api.ApplicationBaseProtocol, org.apache.hadoop.yarn.webapp.View.ViewContext"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="render"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="html" type="org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block"/>
+    </method>
+    <field name="appBaseProt" type="org.apache.hadoop.yarn.api.ApplicationBaseProtocol"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.ContainerBlock -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.WebPageUtils -->
+  <class name="WebPageUtils" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="WebPageUtils"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="appsTableInit" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="appsTableInit" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="isFairSchedulerPage" type="boolean"/>
+    </method>
+    <method name="attemptsTableInit" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="containersTableInit" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.WebPageUtils -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.WebServices -->
+  <class name="WebServices" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="WebServices" type="org.apache.hadoop.yarn.api.ApplicationBaseProtocol"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getApps" return="org.apache.hadoop.yarn.server.webapp.dao.AppsInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="javax.servlet.http.HttpServletRequest"/>
+      <param name="res" type="javax.servlet.http.HttpServletResponse"/>
+      <param name="stateQuery" type="java.lang.String"/>
+      <param name="statesQuery" type="java.util.Set"/>
+      <param name="finalStatusQuery" type="java.lang.String"/>
+      <param name="userQuery" type="java.lang.String"/>
+      <param name="queueQuery" type="java.lang.String"/>
+      <param name="count" type="java.lang.String"/>
+      <param name="startedBegin" type="java.lang.String"/>
+      <param name="startedEnd" type="java.lang.String"/>
+      <param name="finishBegin" type="java.lang.String"/>
+      <param name="finishEnd" type="java.lang.String"/>
+      <param name="applicationTypes" type="java.util.Set"/>
+    </method>
+    <method name="getApp" return="org.apache.hadoop.yarn.server.webapp.dao.AppInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="javax.servlet.http.HttpServletRequest"/>
+      <param name="res" type="javax.servlet.http.HttpServletResponse"/>
+      <param name="appId" type="java.lang.String"/>
+    </method>
+    <method name="getAppAttempts" return="org.apache.hadoop.yarn.server.webapp.dao.AppAttemptsInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="javax.servlet.http.HttpServletRequest"/>
+      <param name="res" type="javax.servlet.http.HttpServletResponse"/>
+      <param name="appId" type="java.lang.String"/>
+    </method>
+    <method name="getAppAttempt" return="org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="javax.servlet.http.HttpServletRequest"/>
+      <param name="res" type="javax.servlet.http.HttpServletResponse"/>
+      <param name="appId" type="java.lang.String"/>
+      <param name="appAttemptId" type="java.lang.String"/>
+    </method>
+    <method name="getContainers" return="org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="javax.servlet.http.HttpServletRequest"/>
+      <param name="res" type="javax.servlet.http.HttpServletResponse"/>
+      <param name="appId" type="java.lang.String"/>
+      <param name="appAttemptId" type="java.lang.String"/>
+    </method>
+    <method name="getContainer" return="org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="javax.servlet.http.HttpServletRequest"/>
+      <param name="res" type="javax.servlet.http.HttpServletResponse"/>
+      <param name="appId" type="java.lang.String"/>
+      <param name="appAttemptId" type="java.lang.String"/>
+      <param name="containerId" type="java.lang.String"/>
+    </method>
+    <method name="init"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="response" type="javax.servlet.http.HttpServletResponse"/>
+    </method>
+    <method name="parseQueries" return="java.util.Set"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="queries" type="java.util.Set"/>
+      <param name="isState" type="boolean"/>
+    </method>
+    <method name="parseApplicationId" return="org.apache.hadoop.yarn.api.records.ApplicationId"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="appId" type="java.lang.String"/>
+    </method>
+    <method name="parseApplicationAttemptId" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="appAttemptId" type="java.lang.String"/>
+    </method>
+    <method name="parseContainerId" return="org.apache.hadoop.yarn.api.records.ContainerId"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="containerId" type="java.lang.String"/>
+    </method>
+    <method name="validateIds"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="appAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+    </method>
+    <method name="getUser" return="org.apache.hadoop.security.UserGroupInformation"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="req" type="javax.servlet.http.HttpServletRequest"/>
+    </method>
+    <field name="appBaseProt" type="org.apache.hadoop.yarn.api.ApplicationBaseProtocol"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.WebServices -->
+</package>
+<package name="org.apache.hadoop.yarn.server.webapp.dao">
+  <!-- start class org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo -->
+  <class name="AppAttemptInfo" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AppAttemptInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AppAttemptInfo" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptReport"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getAppAttemptId" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getHost" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getRpcPort" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getTrackingUrl" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getOriginalTrackingUrl" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getDiagnosticsInfo" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAppAttemptState" return="org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAmContainerId" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="appAttemptId" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="host" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="rpcPort" type="int"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="trackingUrl" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="originalTrackingUrl" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="diagnosticsInfo" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="appAttemptState" type="org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="amContainerId" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.dao.AppAttemptsInfo -->
+  <class name="AppAttemptsInfo" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AppAttemptsInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="add"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="info" type="org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo"/>
+    </method>
+    <method name="getAttempts" return="java.util.ArrayList"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="attempt" type="java.util.ArrayList"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.dao.AppAttemptsInfo -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.dao.AppInfo -->
+  <class name="AppInfo" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AppInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AppInfo" type="org.apache.hadoop.yarn.api.records.ApplicationReport"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getAppId" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getCurrentAppAttemptId" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getUser" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getName" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getQueue" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getType" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getHost" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getRpcPort" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAppState" return="org.apache.hadoop.yarn.api.records.YarnApplicationState"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getProgress" return="float"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getDiagnosticsInfo" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getOriginalTrackingUrl" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getTrackingUrl" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getFinalAppStatus" return="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getSubmittedTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getStartedTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getFinishedTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getElapsedTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getApplicationTags" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="appId" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="currentAppAttemptId" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="user" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="name" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="queue" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="type" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="host" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="rpcPort" type="int"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="appState" type="org.apache.hadoop.yarn.api.records.YarnApplicationState"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="progress" type="float"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="diagnosticsInfo" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="originalTrackingUrl" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="trackingUrl" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="finalAppStatus" type="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="submittedTime" type="long"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="startedTime" type="long"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="finishedTime" type="long"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="elapsedTime" type="long"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="applicationTags" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.dao.AppInfo -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.dao.AppsInfo -->
+  <class name="AppsInfo" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AppsInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="add"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appinfo" type="org.apache.hadoop.yarn.server.webapp.dao.AppInfo"/>
+    </method>
+    <method name="getApps" return="java.util.ArrayList"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="app" type="java.util.ArrayList"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.dao.AppsInfo -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo -->
+  <class name="ContainerInfo" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ContainerInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="ContainerInfo" type="org.apache.hadoop.yarn.api.records.ContainerReport"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getContainerId" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAllocatedMB" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAllocatedVCores" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAssignedNodeId" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getPriority" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getStartedTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getFinishedTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getElapsedTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getDiagnosticsInfo" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprec

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[24/51] [abbrv] hadoop git commit: HDFS-7987. Allow files / directories to be moved (Ravi Prakash via aw)

Posted by vv...@apache.org.
HDFS-7987. Allow files / directories to be moved (Ravi Prakash via aw)


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

Branch: refs/heads/YARN-3926
Commit: d44f4745b4a186dd06dd6837a85ad90a237d7d97
Parents: 0b7b8a3
Author: Allen Wittenauer <aw...@apache.org>
Authored: Fri Jun 10 09:02:28 2016 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Fri Jun 10 09:02:28 2016 -0700

----------------------------------------------------------------------
 .../src/main/webapps/hdfs/explorer.html         |  9 +++
 .../src/main/webapps/hdfs/explorer.js           | 83 ++++++++++++++++----
 .../src/main/webapps/static/hadoop.css          |  7 ++
 3 files changed, 83 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d44f4745/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
index 51f72e5..ad8c374 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
@@ -179,6 +179,13 @@
           data-target="#modal-upload-file" title="Upload Files">
             <span class="glyphicon glyphicon-cloud-upload"></span>
         </button>
+        <button class="btn btn-default dropdown-toggle" type="button"
+          data-toggle="dropdown" title="Cut & Paste">
+        <span class="glyphicon glyphicon-list-alt"></span></button>
+        <ul class="dropdown-menu cut-paste">
+          <li><a id="explorer-cut">Cut</a></li>
+          <li><a id="explorer-paste">Paste</a></li>
+        </ul>
       </div>
     </div>
 
@@ -236,6 +243,7 @@
       <table class="table" id="table-explorer">
         <thead>
           <tr>
+            <th><input type="checkbox" id="file-selector-all"></th>
             <th title="Permissions">Permission</th>
             <th>Owner</th>
             <th>Group</th>
@@ -251,6 +259,7 @@
           {#FileStatus}
           <tr inode-path="{pathSuffix}" data-permission="{permission}"
             class="explorer-entry">
+            <td><input type="checkbox" class="file_selector"> </td>
             <td><span class="explorer-perm-links editable-click">
               {type|helper_to_directory}{permission|helper_to_permission}
               {aclBit|helper_to_acl_bit}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d44f4745/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
index 6fa5f19..1739db2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
@@ -310,22 +310,28 @@
           var absolute_file_path = append_path(current_directory, inode_name);
           delete_path(inode_name, absolute_file_path);
         });
-          
-          $('#table-explorer').dataTable( {
-              'lengthMenu': [ [25, 50, 100, -1], [25, 50, 100, "All"] ],
-              'columns': [
-                  {'searchable': false }, //Permissions
-                  null, //Owner
-                  null, //Group
-                  { 'searchable': false, 'render': func_size_render}, //Size
-                  { 'searchable': false, 'render': func_time_render}, //Last Modified
-                  { 'searchable': false }, //Replication
-                  null, //Block Size
-                  null, //Name
-                  { 'sortable' : false } //Trash
-              ],
-              "deferRender": true
-          });
+
+        $('#file-selector-all').click(function() {
+          $('.file_selector').prop('checked', $('#file-selector-all')[0].checked );
+        });
+
+        //This needs to be last because it repaints the table
+        $('#table-explorer').dataTable( {
+          'lengthMenu': [ [25, 50, 100, -1], [25, 50, 100, "All"] ],
+          'columns': [
+            { 'orderable' : false }, //select
+            {'searchable': false }, //Permissions
+            null, //Owner
+            null, //Group
+            { 'searchable': false, 'render': func_size_render}, //Size
+            { 'searchable': false, 'render': func_time_render}, //Last Modified
+            { 'searchable': false }, //Replication
+            null, //Block Size
+            null, //Name
+            { 'orderable' : false } //Trash
+          ],
+          "deferRender": true
+        });
       });
     }).error(network_error_handler(url));
   }
@@ -417,5 +423,50 @@
     }
   });
 
+  //Store the list of files which have been checked into session storage
+  function store_selected_files(current_directory) {
+    sessionStorage.setItem("source_directory", current_directory);
+    var selected_files = $("input:checked.file_selector");
+    var selected_file_names = new Array();
+    selected_files.each(function(index) {
+      selected_file_names[index] = $(this).closest('tr').attr('inode-path');
+    })
+    sessionStorage.setItem("selected_file_names", JSON.stringify(selected_file_names));
+    alert("Cut " + selected_file_names.length + " files/directories");
+  }
+
+  //Retrieve the list of files from session storage and rename them to the current
+  //directory
+  function paste_selected_files() {
+    var files = JSON.parse(sessionStorage.getItem("selected_file_names"));
+    var source_directory = sessionStorage.getItem("source_directory");
+    $.each(files, function(index, value) {
+      var url = "/webhdfs/v1"
+        + encode_path(append_path(source_directory, value))
+        + '?op=RENAME&destination='
+        + encode_path(append_path(current_directory, value));
+      $.ajax({
+        type: 'PUT',
+        url: url
+      }).done(function(data) {
+        if(index == files.length - 1) {
+          browse_directory(current_directory);
+        }
+      }).error(function(jqXHR, textStatus, errorThrown) {
+        show_err_msg("Couldn't move file " + value + ". " + errorThrown);
+      });
+
+    })
+  }
+
+  $('#explorer-cut').click(function() {
+    store_selected_files(current_directory);
+  });
+
+  $('#explorer-paste').click(function() {
+    paste_selected_files();
+  });
+
+
   init();
 })();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d44f4745/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
index 2ed5f29..5021fb5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
@@ -285,4 +285,11 @@ header.bs-docs-nav, header.bs-docs-nav .navbar-brand {
     margin-bottom: 0;
     font-weight: normal;
     cursor: pointer;
+}
+
+.cut-paste {
+  width: 75px;
+  min-width: 75px;
+  float: right;
+  left: 75px;
 }
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[13/51] [abbrv] hadoop git commit: YARN-4308. ContainersAggregated CPU resource utilization reports negative usage in first few heartbeats. Contributed by Sunil G

Posted by vv...@apache.org.
YARN-4308. ContainersAggregated CPU resource utilization reports negative usage in first few heartbeats. Contributed by Sunil G


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

Branch: refs/heads/YARN-3926
Commit: 1500a0a3009e453c9f05a93df7a78b4e185eef30
Parents: ae04765
Author: Naganarasimha <na...@apache.org>
Authored: Thu Jun 9 05:41:09 2016 +0530
Committer: Naganarasimha <na...@apache.org>
Committed: Thu Jun 9 05:41:09 2016 +0530

----------------------------------------------------------------------
 .../yarn/util/ProcfsBasedProcessTree.java       |  8 +++
 .../util/ResourceCalculatorProcessTree.java     |  4 +-
 .../yarn/util/WindowsBasedProcessTree.java      |  8 +++
 .../monitor/ContainersMonitorImpl.java          |  9 +++
 .../MockCPUResourceCalculatorProcessTree.java   | 70 ++++++++++++++++++++
 .../MockResourceCalculatorProcessTree.java      |  5 ++
 .../TestContainersMonitorResourceChange.java    | 62 ++++++++++++++++-
 7 files changed, 163 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1500a0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
index bb9c183..80d49c3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
@@ -467,6 +467,14 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
     return totalStime.add(BigInteger.valueOf(totalUtime));
   }
 
+  /**
+   * Get the CPU usage by all the processes in the process-tree in Unix.
+   * Note: UNAVAILABLE will be returned in case when CPU usage is not
+   * available. It is NOT advised to return any other error code.
+   *
+   * @return percentage CPU usage since the process-tree was created,
+   * {@link #UNAVAILABLE} if CPU usage cannot be calculated or not available.
+   */
   @Override
   public float getCpuUsagePercent() {
     BigInteger processTotalJiffies = getTotalProcessJiffies();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1500a0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java
index 7214c75..771ec86 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java
@@ -187,9 +187,11 @@ public abstract class ResourceCalculatorProcessTree extends Configured {
    * Get the CPU usage by all the processes in the process-tree based on
    * average between samples as a ratio of overall CPU cycles similar to top.
    * Thus, if 2 out of 4 cores are used this should return 200.0.
+   * Note: UNAVAILABLE will be returned in case when CPU usage is not
+   * available. It is NOT advised to return any other error code.
    *
    * @return percentage CPU usage since the process-tree was created,
-   * {@link #UNAVAILABLE} if it cannot be calculated.
+   * {@link #UNAVAILABLE} if CPU usage cannot be calculated or not available.
    */
   public float getCpuUsagePercent() {
     return UNAVAILABLE;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1500a0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java
index 7858292..1c7eaf7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java
@@ -268,6 +268,14 @@ public class WindowsBasedProcessTree extends ResourceCalculatorProcessTree {
     return BigInteger.valueOf(totalMs);
   }
 
+  /**
+   * Get the CPU usage by all the processes in the process-tree in Windows.
+   * Note: UNAVAILABLE will be returned in case when CPU usage is not
+   * available. It is NOT advised to return any other error code.
+   *
+   * @return percentage CPU usage since the process-tree was created,
+   * {@link #UNAVAILABLE} if CPU usage cannot be calculated or not available.
+   */
   @Override
   public float getCpuUsagePercent() {
     BigInteger processTotalMs = getTotalProcessMs();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1500a0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
index cfe6f801..b5c2747 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
@@ -455,6 +455,15 @@ public class ContainersMonitorImpl extends AbstractService implements
             // cpuUsagePercentPerCore should be 300% and
             // cpuUsageTotalCoresPercentage should be 50%
             float cpuUsagePercentPerCore = pTree.getCpuUsagePercent();
+            if (cpuUsagePercentPerCore < 0) {
+              // CPU usage is not available likely because the container just
+              // started. Let us skip this turn and consider this container
+              // in the next iteration.
+              LOG.info("Skipping monitoring container " + containerId
+                  + " since CPU usage is not yet available.");
+              continue;
+            }
+
             float cpuUsageTotalCoresPercentage = cpuUsagePercentPerCore /
                 resourceCalculatorPlugin.getNumProcessors();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1500a0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockCPUResourceCalculatorProcessTree.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockCPUResourceCalculatorProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockCPUResourceCalculatorProcessTree.java
new file mode 100644
index 0000000..eb35c91
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockCPUResourceCalculatorProcessTree.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor;
+
+import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
+
+/**
+ * Mock class to obtain resource usage (CPU).
+ */
+public class MockCPUResourceCalculatorProcessTree
+    extends ResourceCalculatorProcessTree {
+
+  private long cpuPercentage = ResourceCalculatorProcessTree.UNAVAILABLE;
+
+  /**
+   * Constructor for MockCPUResourceCalculatorProcessTree with specified root
+   * process.
+   * @param root
+   */
+  public MockCPUResourceCalculatorProcessTree(String root) {
+    super(root);
+  }
+
+  @Override
+  public void updateProcessTree() {
+  }
+
+  @Override
+  public String getProcessTreeDump() {
+    return "";
+  }
+
+  @Override
+  public long getCumulativeCpuTime() {
+    return 0;
+  }
+
+  @Override
+  public boolean checkPidPgrpidForMatch() {
+    return true;
+  }
+
+  @Override
+  public float getCpuUsagePercent() {
+    long cpu = this.cpuPercentage;
+    // First getter call will be returned with -1, and other calls will
+    // return non-zero value as defined below.
+    if (cpu == ResourceCalculatorProcessTree.UNAVAILABLE) {
+      // Set a default value other than 0 for test.
+      this.cpuPercentage = 50;
+    }
+    return cpu;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1500a0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorProcessTree.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorProcessTree.java
index c5aaa77..ff2a570 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorProcessTree.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorProcessTree.java
@@ -54,4 +54,9 @@ public class MockResourceCalculatorProcessTree extends ResourceCalculatorProcess
   public long getRssMemorySize() {
     return this.rssMemorySize;
   }
+
+  @Override
+  public float getCpuUsagePercent() {
+    return 0;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1500a0a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.java
index 1a0c690..2df0c98 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.java
@@ -27,8 +27,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceUtilization;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -43,19 +43,21 @@ import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
 import org.apache.hadoop.yarn.server.nodemanager.executor.DeletionAsUserContext;
 import org.apache.hadoop.yarn.server.nodemanager.executor.LocalizerStartContext;
+import org.apache.log4j.Logger;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertFalse;
 
 public class TestContainersMonitorResourceChange {
 
+  static final Logger LOG = Logger
+      .getLogger(TestContainersMonitorResourceChange.class);
   private ContainersMonitorImpl containersMonitor;
   private MockExecutor executor;
   private Configuration conf;
@@ -63,6 +65,8 @@ public class TestContainersMonitorResourceChange {
   private Context context;
   private MockContainerEventHandler containerEventHandler;
 
+  static final int WAIT_MS_PER_LOOP = 20; // 20 milli seconds
+
   private static class MockExecutor extends ContainerExecutor {
     @Override
     public void init() throws IOException {
@@ -232,6 +236,60 @@ public class TestContainersMonitorResourceChange {
     containersMonitor.stop();
   }
 
+  @Test
+  public void testContainersCPUResourceForDefaultValue() throws Exception {
+    Configuration newConf = new Configuration(conf);
+    // set container monitor interval to be 20s
+    newConf.setLong(YarnConfiguration.NM_CONTAINER_MON_INTERVAL_MS, 20L);
+    containersMonitor = createContainersMonitor(executor, dispatcher, context);
+    newConf.set(YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE,
+        MockCPUResourceCalculatorProcessTree.class.getCanonicalName());
+    // set container monitor interval to be 20ms
+    containersMonitor.init(newConf);
+    containersMonitor.start();
+
+    // create container 1
+    containersMonitor.handle(new ContainerStartMonitoringEvent(
+        getContainerId(1), 2100L, 1000L, 1, 0, 0));
+
+    // Verify the container utilization value.
+    // Since MockCPUResourceCalculatorProcessTree will return a -1 as CPU
+    // utilization, containersUtilization will not be calculated and hence it
+    // will be 0.
+    assertEquals(
+        "Resource utilization must be default with MonitorThread's first run",
+        0, containersMonitor.getContainersUtilization()
+            .compareTo(ResourceUtilization.newInstance(0, 0, 0.0f)));
+
+    // Verify the container utilization value. Since atleast one round is done,
+    // we can expect a non-zero value for container utilization as
+    // MockCPUResourceCalculatorProcessTree#getCpuUsagePercent will return 50.
+    waitForContainerResourceUtilizationChange(containersMonitor, 100);
+
+    containersMonitor.stop();
+  }
+
+  public static void waitForContainerResourceUtilizationChange(
+      ContainersMonitorImpl containersMonitor, int timeoutMsecs)
+      throws InterruptedException {
+    int timeWaiting = 0;
+    while (0 == containersMonitor.getContainersUtilization()
+        .compareTo(ResourceUtilization.newInstance(0, 0, 0.0f))) {
+      if (timeWaiting >= timeoutMsecs) {
+        break;
+      }
+
+      LOG.info(
+          "Monitor thread is waiting for resource utlization change.");
+      Thread.sleep(WAIT_MS_PER_LOOP);
+      timeWaiting += WAIT_MS_PER_LOOP;
+    }
+
+    assertTrue("Resource utilization is not changed from second run onwards",
+        0 != containersMonitor.getContainersUtilization()
+            .compareTo(ResourceUtilization.newInstance(0, 0, 0.0f)));
+  }
+
   private ContainersMonitorImpl createContainersMonitor(
       ContainerExecutor containerExecutor, AsyncDispatcher dispatcher,
       Context context) {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[34/51] [abbrv] hadoop git commit: YARN-3426. Add jdiff support to YARN. (vinodkv via wangda)

Posted by vv...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/03fc6b1b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_API_2.6.0.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_API_2.6.0.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_API_2.6.0.xml
new file mode 100644
index 0000000..5d58600
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_API_2.6.0.xml
@@ -0,0 +1,13076 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!--
+   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.
+-->
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Wed Apr 08 11:29:43 PDT 2015 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="hadoop-yarn-api 2.6.0"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.ExcludePrivateAnnotationsJDiffDoclet -docletpath /Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/hadoop-annotations.jar:/Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/jdiff.jar -verbose -classpath /Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/classes:/Users/llu/.m2/repository/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/Users/llu/.m2/repository/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/Users/llu/.m2/repository/com/google/code/findbugs/jsr305/1.3.9/jsr305-1.3.9.jar:/Users/llu/.m2/repository/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/Users/llu/.m2/repository/org/apache/hadoop/hadoop-common/2.6.0/hadoop-common-2.6.0.jar:/Users/llu/.m2/repository/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/Users/llu/.m2/repository/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/Users/l
 lu/.m2/repository/xmlenc/xmlenc/0.52/xmlenc-0.52.jar:/Users/llu/.m2/repository/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1.jar:/Users/llu/.m2/repository/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/Users/llu/.m2/repository/commons-io/commons-io/2.4/commons-io-2.4.jar:/Users/llu/.m2/repository/commons-net/commons-net/3.1/commons-net-3.1.jar:/Users/llu/.m2/repository/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1.jar:/Users/llu/.m2/repository/javax/servlet/servlet-api/2.5/servlet-api-2.5.jar:/Users/llu/.m2/repository/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/Users/llu/.m2/repository/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/Users/llu/.m2/repository/com/sun/jersey/jersey-core/1.9/jersey-core-1.9.jar:/Users/llu/.m2/repository/com/sun/jersey/jersey-json/1.9/jersey-json-1.9.jar:/Users/llu/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/Users/llu/.m2/repository/com/sun/xml/bind/jaxb-impl/2.2
 .3-1/jaxb-impl-2.2.3-1.jar:/Users/llu/.m2/repository/javax/xml/bind/jaxb-api/2.2.2/jaxb-api-2.2.2.jar:/Users/llu/.m2/repository/javax/xml/stream/stax-api/1.0-2/stax-api-1.0-2.jar:/Users/llu/.m2/repository/javax/activation/activation/1.1/activation-1.1.jar:/Users/llu/.m2/repository/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/Users/llu/.m2/repository/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/Users/llu/.m2/repository/com/sun/jersey/jersey-server/1.9/jersey-server-1.9.jar:/Users/llu/.m2/repository/asm/asm/3.2/asm-3.2.jar:/Users/llu/.m2/repository/javax/servlet/jsp/jsp-api/2.1/jsp-api-2.1.jar:/Users/llu/.m2/repository/log4j/log4j/1.2.17/log4j-1.2.17.jar:/Users/llu/.m2/repository/net/java/dev/jets3t/jets3t/0.9.0/jets3t-0.9.0.jar:/Users/llu/.m2/repository/org/apache/httpcomponents/httpclient/4.2.5/httpclient-4.2.5.jar:/Users/llu/.m2/repository/org/apache/httpcomponents/httpcore/4.2.5/httpcore-4.2.5.jar:/Users/llu/.m2/repository/com/jamesmurty/uti
 ls/java-xmlbuilder/0.4/java-xmlbuilder-0.4.jar:/Users/llu/.m2/repository/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/Users/llu/.m2/repository/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/Users/llu/.m2/repository/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/Users/llu/.m2/repository/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar:/Users/llu/.m2/repository/org/slf4j/slf4j-api/1.7.5/slf4j-api-1.7.5.jar:/Users/llu/.m2/repository/org/slf4j/slf4j-log4j12/1.7.5/slf4j-log4j12-1.7.5.jar:/Users/llu/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/llu/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/llu/.m2/repository/org/apache/avro/avro/1.7.4/avro-1.7.4.jar:/Users/llu/.m2/repository/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/Users/llu/.m2/repository/org/xerial/snappy/snap
 py-java/1.0.4.1/snappy-java-1.0.4.1.jar:/Users/llu/.m2/repository/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/Users/llu/.m2/repository/org/apache/hadoop/hadoop-auth/2.6.0/hadoop-auth-2.6.0.jar:/Users/llu/.m2/repository/org/apache/directory/server/apacheds-kerberos-codec/2.0.0-M15/apacheds-kerberos-codec-2.0.0-M15.jar:/Users/llu/.m2/repository/org/apache/directory/server/apacheds-i18n/2.0.0-M15/apacheds-i18n-2.0.0-M15.jar:/Users/llu/.m2/repository/org/apache/directory/api/api-asn1-api/1.0.0-M20/api-asn1-api-1.0.0-M20.jar:/Users/llu/.m2/repository/org/apache/directory/api/api-util/1.0.0-M20/api-util-1.0.0-M20.jar:/Users/llu/.m2/repository/org/apache/curator/curator-framework/2.6.0/curator-framework-2.6.0.jar:/Users/llu/.m2/repository/com/jcraft/jsch/0.1.42/jsch-0.1.42.jar:/Users/llu/.m2/repository/org/apache/curator/curator-client/2.6.0/curator-client-2.6.0.jar:/Users/llu/.m2/repository/org/apache/curator/curator-recipes/2.6.0/curator-recipes-2.6.0.jar:/Users/llu/.m2/repository/or
 g/htrace/htrace-core/3.0.4/htrace-core-3.0.4.jar:/Users/llu/.m2/repository/org/apache/zookeeper/zookeeper/3.4.6/zookeeper-3.4.6.jar:/Users/llu/.m2/repository/io/netty/netty/3.6.2.Final/netty-3.6.2.Final.jar:/Users/llu/.m2/repository/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/Users/llu/.m2/repository/org/tukaani/xz/1.0/xz-1.0.jar:/Users/llu/.m2/repository/org/apache/hadoop/hadoop-annotations/2.6.0/hadoop-annotations-2.6.0.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/lib/tools.jar:/Users/llu/.m2/repository/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar -sourcepath /Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java -apidir /Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/site/jdiff/xml -apiname hadoop-yarn-api 2.6.0 -->
+<package name="org.apache.hadoop.yarn.api">
+  <!-- start interface org.apache.hadoop.yarn.api.ApplicationClientProtocol -->
+  <interface name="ApplicationClientProtocol"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getNewApplication" return="org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by clients to obtain a new {@link ApplicationId} for
+ submitting new applications.</p>
+
+ <p>The <code>ResourceManager</code> responds with a new, monotonically
+ increasing, {@link ApplicationId} which is used by the client to submit
+ a new application.</p>
+
+ <p>The <code>ResourceManager</code> also responds with details such
+ as maximum resource capabilities in the cluster as specified in
+ {@link GetNewApplicationResponse}.</p>
+
+ @param request request to get a new <code>ApplicationId</code>
+ @return response containing the new <code>ApplicationId</code> to be used
+ to submit an application
+ @throws YarnException
+ @throws IOException
+ @see #submitApplication(SubmitApplicationRequest)]]>
+      </doc>
+    </method>
+    <method name="submitApplication" return="org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by clients to submit a new application to the
+ <code>ResourceManager.</code></p>
+
+ <p>The client is required to provide details such as queue,
+ {@link Resource} required to run the <code>ApplicationMaster</code>,
+ the equivalent of {@link ContainerLaunchContext} for launching
+ the <code>ApplicationMaster</code> etc. via the
+ {@link SubmitApplicationRequest}.</p>
+
+ <p>Currently the <code>ResourceManager</code> sends an immediate (empty)
+ {@link SubmitApplicationResponse} on accepting the submission and throws
+ an exception if it rejects the submission. However, this call needs to be
+ followed by {@link #getApplicationReport(GetApplicationReportRequest)}
+ to make sure that the application gets properly submitted - obtaining a
+ {@link SubmitApplicationResponse} from ResourceManager doesn't guarantee
+ that RM 'remembers' this application beyond failover or restart. If RM
+ failover or RM restart happens before ResourceManager saves the
+ application's state successfully, the subsequent
+ {@link #getApplicationReport(GetApplicationReportRequest)} will throw
+ a {@link ApplicationNotFoundException}. The Clients need to re-submit
+ the application with the same {@link ApplicationSubmissionContext} when
+ it encounters the {@link ApplicationNotFoundException} on the
+ {@link #getApplicationReport(GetApplicationReportRequest)} call.</p>
+
+ <p>During the submission process, it checks whether the application
+ already exists. If the application exists, it will simply return
+ SubmitApplicationResponse</p>
+
+ <p> In secure mode,the <code>ResourceManager</code> verifies access to
+ queues etc. before accepting the application submission.</p>
+
+ @param request request to submit a new application
+ @return (empty) response on accepting the submission
+ @throws YarnException
+ @throws IOException
+ @throws InvalidResourceRequestException
+           The exception is thrown when a {@link ResourceRequest} is out of
+           the range of the configured lower and upper resource boundaries.
+ @see #getNewApplication(GetNewApplicationRequest)]]>
+      </doc>
+    </method>
+    <method name="forceKillApplication" return="org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by clients to request the
+ <code>ResourceManager</code> to abort submitted application.</p>
+
+ <p>The client, via {@link KillApplicationRequest} provides the
+ {@link ApplicationId} of the application to be aborted.</p>
+
+ <p> In secure mode,the <code>ResourceManager</code> verifies access to the
+ application, queue etc. before terminating the application.</p>
+
+ <p>Currently, the <code>ResourceManager</code> returns an empty response
+ on success and throws an exception on rejecting the request.</p>
+
+ @param request request to abort a submitted application
+ @return <code>ResourceManager</code> returns an empty response
+         on success and throws an exception on rejecting the request
+ @throws YarnException
+ @throws IOException
+ @see #getQueueUserAcls(GetQueueUserAclsInfoRequest)]]>
+      </doc>
+    </method>
+    <method name="getApplicationReport" return="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by clients to get a report of an Application from
+ the <code>ResourceManager</code>.</p>
+
+ <p>The client, via {@link GetApplicationReportRequest} provides the
+ {@link ApplicationId} of the application.</p>
+
+ <p> In secure mode,the <code>ResourceManager</code> verifies access to the
+ application, queue etc. before accepting the request.</p>
+
+ <p>The <code>ResourceManager</code> responds with a
+ {@link GetApplicationReportResponse} which includes the
+ {@link ApplicationReport} for the application.</p>
+
+ <p>If the user does not have <code>VIEW_APP</code> access then the
+ following fields in the report will be set to stubbed values:
+ <ul>
+   <li>host - set to "N/A"</li>
+   <li>RPC port - set to -1</li>
+   <li>client token - set to "N/A"</li>
+   <li>diagnostics - set to "N/A"</li>
+   <li>tracking URL - set to "N/A"</li>
+   <li>original tracking URL - set to "N/A"</li>
+   <li>resource usage report - all values are -1</li>
+ </ul></p>
+
+ @param request request for an application report
+ @return application report
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getClusterMetrics" return="org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by clients to get metrics about the cluster from
+ the <code>ResourceManager</code>.</p>
+
+ <p>The <code>ResourceManager</code> responds with a
+ {@link GetClusterMetricsResponse} which includes the
+ {@link YarnClusterMetrics} with details such as number of current
+ nodes in the cluster.</p>
+
+ @param request request for cluster metrics
+ @return cluster metrics
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by clients to get a report of Applications
+ matching the filters defined by {@link GetApplicationsRequest}
+ in the cluster from the <code>ResourceManager</code>.</p>
+
+ <p>The <code>ResourceManager</code> responds with a
+ {@link GetApplicationsResponse} which includes the
+ {@link ApplicationReport} for the applications.</p>
+
+ <p>If the user does not have <code>VIEW_APP</code> access for an
+ application then the corresponding report will be filtered as
+ described in {@link #getApplicationReport(GetApplicationReportRequest)}.
+ </p>
+
+ @param request request for report on applications
+ @return report on applications matching the given application types
+           defined in the request
+ @throws YarnException
+ @throws IOException
+ @see GetApplicationsRequest]]>
+      </doc>
+    </method>
+    <method name="getClusterNodes" return="org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by clients to get a report of all nodes
+ in the cluster from the <code>ResourceManager</code>.</p>
+
+ <p>The <code>ResourceManager</code> responds with a
+ {@link GetClusterNodesResponse} which includes the
+ {@link NodeReport} for all the nodes in the cluster.</p>
+
+ @param request request for report on all nodes
+ @return report on all nodes
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getQueueInfo" return="org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by clients to get information about <em>queues</em>
+ from the <code>ResourceManager</code>.</p>
+
+ <p>The client, via {@link GetQueueInfoRequest}, can ask for details such
+ as used/total resources, child queues, running applications etc.</p>
+
+ <p> In secure mode,the <code>ResourceManager</code> verifies access before
+ providing the information.</p>
+
+ @param request request to get queue information
+ @return queue information
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getQueueUserAcls" return="org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by clients to get information about <em>queue
+ acls</em> for <em>current user</em> from the <code>ResourceManager</code>.
+ </p>
+
+ <p>The <code>ResourceManager</code> responds with queue acls for all
+ existing queues.</p>
+
+ @param request request to get queue acls for <em>current user</em>
+ @return queue acls for <em>current user</em>
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getDelegationToken" return="org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by clients to get delegation token, enabling the
+ containers to be able to talk to the service using those tokens.
+
+  <p> The <code>ResourceManager</code> responds with the delegation
+  {@link Token} that can be used by the client to speak to this
+  service.
+ @param request request to get a delegation token for the client.
+ @return delegation token that can be used to talk to this service
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="moveApplicationAcrossQueues" return="org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Move an application to a new queue.
+
+ @param request the application ID and the target queue
+ @return an empty response
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttemptReport" return="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to get a report of an Application Attempt
+ from the <code>ResourceManager</code>
+ </p>
+
+ <p>
+ The client, via {@link GetApplicationAttemptReportRequest} provides the
+ {@link ApplicationAttemptId} of the application attempt.
+ </p>
+
+ <p>
+ In secure mode,the <code>ResourceManager</code> verifies access to
+ the method before accepting the request.
+ </p>
+
+ <p>
+ The <code>ResourceManager</code> responds with a
+ {@link GetApplicationAttemptReportResponse} which includes the
+ {@link ApplicationAttemptReport} for the application attempt.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access then the following
+ fields in the report will be set to stubbed values:
+ <ul>
+ <li>host</li>
+ <li>RPC port</li>
+ <li>client token</li>
+ <li>diagnostics - set to "N/A"</li>
+ <li>tracking URL</li>
+ </ul>
+ </p>
+
+ @param request
+          request for an application attempt report
+ @return application attempt report
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttempts" return="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to get a report of all Application attempts
+ in the cluster from the <code>ResourceManager</code>
+ </p>
+
+ <p>
+ The <code>ResourceManager</code> responds with a
+ {@link GetApplicationAttemptsRequest} which includes the
+ {@link ApplicationAttemptReport} for all the applications attempts of a
+ specified application attempt.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationAttemptReport(GetApplicationAttemptReportRequest)}.
+ </p>
+
+ @param request
+          request for reports on all application attempts of an application
+ @return reports on all application attempts of an application
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainerReport" return="org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to get a report of an Container from the
+ <code>ResourceManager</code>
+ </p>
+
+ <p>
+ The client, via {@link GetContainerReportRequest} provides the
+ {@link ContainerId} of the container.
+ </p>
+
+ <p>
+ In secure mode,the <code>ResourceManager</code> verifies access to the
+ method before accepting the request.
+ </p>
+
+ <p>
+ The <code>ResourceManager</code> responds with a
+ {@link GetContainerReportResponse} which includes the
+ {@link ContainerReport} for the container.
+ </p>
+
+ @param request
+          request for a container report
+ @return container report
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainers" return="org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to get a report of Containers for an
+ application attempt from the <code>ResourceManager</code>
+ </p>
+
+ <p>
+ The client, via {@link GetContainersRequest} provides the
+ {@link ApplicationAttemptId} of the application attempt.
+ </p>
+
+ <p>
+ In secure mode,the <code>ResourceManager</code> verifies access to the
+ method before accepting the request.
+ </p>
+
+ <p>
+ The <code>ResourceManager</code> responds with a
+ {@link GetContainersResponse} which includes a list of
+ {@link ContainerReport} for all the containers of a specific application
+ attempt.
+ </p>
+
+ @param request
+          request for a list of container reports of an application attempt.
+ @return reports on all containers of an application attempt
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="submitReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to submit a new reservation to the
+ {@code ResourceManager}.
+ </p>
+
+ <p>
+ The client packages all details of its request in a
+ {@link ReservationSubmissionRequest} object. This contains information
+ about the amount of capacity, temporal constraints, and concurrency needs.
+ Furthermore, the reservation might be composed of multiple stages, with
+ ordering dependencies among them.
+ </p>
+
+ <p>
+ In order to respond, a new admission control component in the
+ {@code ResourceManager} performs an analysis of the resources that have
+ been committed over the period of time the user is requesting, verify that
+ the user requests can be fulfilled, and that it respect a sharing policy
+ (e.g., {@code CapacityOverTimePolicy}). Once it has positively determined
+ that the ReservationSubmissionRequest is satisfiable the
+ {@code ResourceManager} answers with a
+ {@link ReservationSubmissionResponse} that include a non-null
+ {@link ReservationId}. Upon failure to find a valid allocation the response
+ is an exception with the reason.
+
+ On application submission the client can use this {@link ReservationId} to
+ obtain access to the reserved resources.
+ </p>
+
+ <p>
+ The system guarantees that during the time-range specified by the user, the
+ reservationID will be corresponding to a valid reservation. The amount of
+ capacity dedicated to such queue can vary overtime, depending of the
+ allocation that has been determined. But it is guaranteed to satisfy all
+ the constraint expressed by the user in the
+ {@link ReservationSubmissionRequest}.
+ </p>
+
+ @param request the request to submit a new Reservation
+ @return response the {@link ReservationId} on accepting the submission
+ @throws YarnException if the request is invalid or reservation cannot be
+           created successfully
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="updateReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to update an existing Reservation. This is
+ referred to as a re-negotiation process, in which a user that has
+ previously submitted a Reservation.
+ </p>
+
+ <p>
+ The allocation is attempted by virtually substituting all previous
+ allocations related to this Reservation with new ones, that satisfy the new
+ {@link ReservationUpdateRequest}. Upon success the previous allocation is
+ substituted by the new one, and on failure (i.e., if the system cannot find
+ a valid allocation for the updated request), the previous allocation
+ remains valid.
+
+ The {@link ReservationId} is not changed, and applications currently
+ running within this reservation will automatically receive the resources
+ based on the new allocation.
+ </p>
+
+ @param request to update an existing Reservation (the ReservationRequest
+          should refer to an existing valid {@link ReservationId})
+ @return response empty on successfully updating the existing reservation
+ @throws YarnException if the request is invalid or reservation cannot be
+           updated successfully
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="deleteReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to remove an existing Reservation.
+
+ Upon deletion of a reservation applications running with this reservation,
+ are automatically downgraded to normal jobs running without any dedicated
+ reservation.
+ </p>
+
+ @param request to remove an existing Reservation (the ReservationRequest
+          should refer to an existing valid {@link ReservationId})
+ @return response empty on successfully deleting the existing reservation
+ @throws YarnException if the request is invalid or reservation cannot be
+           deleted successfully
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getNodeToLabels" return="org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by client to get node to labels mappings in existing cluster
+ </p>
+
+ @param request
+ @return node to labels mappings
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getClusterNodeLabels" return="org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by client to get node labels in the cluster
+ </p>
+
+ @param request to get node labels collection of this cluster
+ @return node labels collection of this cluster
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>The protocol between clients and the <code>ResourceManager</code>
+ to submit/abort jobs and to get information on applications, cluster metrics,
+ nodes, queues and ACLs.</p>]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.api.ApplicationClientProtocol -->
+  <!-- start interface org.apache.hadoop.yarn.api.ApplicationConstants -->
+  <interface name="ApplicationConstants"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <field name="APP_SUBMIT_TIME_ENV" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[The environment variable for APP_SUBMIT_TIME. Set in AppMaster environment
+ only]]>
+      </doc>
+    </field>
+    <field name="CONTAINER_TOKEN_FILE_ENV_NAME" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[The cache file into which container token is written]]>
+      </doc>
+    </field>
+    <field name="APPLICATION_WEB_PROXY_BASE_ENV" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[The environmental variable for APPLICATION_WEB_PROXY_BASE. Set in
+ ApplicationMaster's environment only. This states that for all non-relative
+ web URLs in the app masters web UI what base should they have.]]>
+      </doc>
+    </field>
+    <field name="LOG_DIR_EXPANSION_VAR" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[The temporary environmental variable for container log directory. This
+ should be replaced by real container log directory on container launch.]]>
+      </doc>
+    </field>
+    <field name="CLASS_PATH_SEPARATOR" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[This constant is used to construct class path and it will be replaced with
+ real class path separator(':' for Linux and ';' for Windows) by
+ NodeManager on container launch. User has to use this constant to construct
+ class path if user wants cross-platform practice i.e. submit an application
+ from a Windows client to a Linux/Unix server or vice versa.]]>
+      </doc>
+    </field>
+    <field name="PARAMETER_EXPANSION_LEFT" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[The following two constants are used to expand parameter and it will be
+ replaced with real parameter expansion marker ('%' for Windows and '$' for
+ Linux) by NodeManager on container launch. For example: {{VAR}} will be
+ replaced as $VAR on Linux, and %VAR% on Windows. User has to use this
+ constant to construct class path if user wants cross-platform practice i.e.
+ submit an application from a Windows client to a Linux/Unix server or vice
+ versa.]]>
+      </doc>
+    </field>
+    <field name="PARAMETER_EXPANSION_RIGHT" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[User has to use this constant to construct class path if user wants
+ cross-platform practice i.e. submit an application from a Windows client to
+ a Linux/Unix server or vice versa.]]>
+      </doc>
+    </field>
+    <field name="STDERR" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="STDOUT" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="MAX_APP_ATTEMPTS_ENV" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[The environment variable for MAX_APP_ATTEMPTS. Set in AppMaster environment
+ only]]>
+      </doc>
+    </field>
+    <doc>
+    <![CDATA[This is the API for the applications comprising of constants that YARN sets
+ up for the applications and the containers.
+
+ TODO: Investigate the semantics and security of each cross-boundary refs.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.api.ApplicationConstants -->
+  <!-- start class org.apache.hadoop.yarn.api.ApplicationConstants.Environment -->
+  <class name="ApplicationConstants.Environment" extends="java.lang.Enum"
+    abstract="false"
+    static="true" final="true" visibility="public"
+    deprecated="not deprecated">
+    <method name="values" return="org.apache.hadoop.yarn.api.ApplicationConstants.Environment[]"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="valueOf" return="org.apache.hadoop.yarn.api.ApplicationConstants.Environment"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+    </method>
+    <method name="key" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="$" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Expand the environment variable based on client OS environment variable
+ expansion syntax (e.g. $VAR for Linux and %VAR% for Windows).
+ <p>
+ Note: Use $$() method for cross-platform practice i.e. submit an
+ application from a Windows client to a Linux/Unix server or vice versa.
+ </p>]]>
+      </doc>
+    </method>
+    <method name="$$" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Expand the environment variable in platform-agnostic syntax. The
+ parameter expansion marker "{{VAR}}" will be replaced with real parameter
+ expansion marker ('%' for Windows and '$' for Linux) by NodeManager on
+ container launch. For example: {{VAR}} will be replaced as $VAR on Linux,
+ and %VAR% on Windows.]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[Environment for Applications.
+
+ Some of the environment variables for applications are <em>final</em>
+ i.e. they cannot be modified by the applications.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.api.ApplicationConstants.Environment -->
+  <!-- start interface org.apache.hadoop.yarn.api.ApplicationHistoryProtocol -->
+  <interface name="ApplicationHistoryProtocol"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getApplicationReport" return="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to get a report of an Application from the
+ <code>ResourceManager</code>.
+ </p>
+
+ <p>
+ The client, via {@link GetApplicationReportRequest} provides the
+ {@link ApplicationId} of the application.
+ </p>
+
+ <p>
+ In secure mode,the <code>ApplicationHistoryServer</code> verifies access to
+ the application, queue etc. before accepting the request.
+ </p>
+
+ <p>
+ The <code>ApplicationHistoryServer</code> responds with a
+ {@link GetApplicationReportResponse} which includes the
+ {@link ApplicationReport} for the application.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access then the following
+ fields in the report will be set to stubbed values:
+ <ul>
+ <li>host - set to "N/A"</li>
+ <li>RPC port - set to -1</li>
+ <li>client token - set to "N/A"</li>
+ <li>diagnostics - set to "N/A"</li>
+ <li>tracking URL - set to "N/A"</li>
+ <li>original tracking URL - set to "N/A"</li>
+ <li>resource usage report - all values are -1</li>
+ </ul>
+ </p>
+
+ @param request
+          request for an application report
+ @return application report
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to get a report of all Applications in the
+ cluster from the <code>ApplicationHistoryServer</code>.
+ </p>
+
+ <p>
+ The <code>ApplicationHistoryServer</code> responds with a
+ {@link GetApplicationsResponse} which includes a list of
+ {@link ApplicationReport} for all the applications.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(GetApplicationReportRequest)}.
+ </p>
+
+ @param request
+          request for reports on all the applications
+ @return report on applications matching the given application types defined
+         in the request
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttemptReport" return="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to get a report of an Application Attempt
+ from the <code>ApplicationHistoryServer</code>.
+ </p>
+
+ <p>
+ The client, via {@link GetApplicationAttemptReportRequest} provides the
+ {@link ApplicationAttemptId} of the application attempt.
+ </p>
+
+ <p>
+ In secure mode,the <code>ApplicationHistoryServer</code> verifies access to
+ the method before accepting the request.
+ </p>
+
+ <p>
+ The <code>ApplicationHistoryServer</code> responds with a
+ {@link GetApplicationAttemptReportResponse} which includes the
+ {@link ApplicationAttemptReport} for the application attempt.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access then the following
+ fields in the report will be set to stubbed values:
+ <ul>
+ <li>host</li>
+ <li>RPC port</li>
+ <li>client token</li>
+ <li>diagnostics - set to "N/A"</li>
+ <li>tracking URL</li>
+ </ul>
+ </p>
+
+ @param request
+          request for an application attempt report
+ @return application attempt report
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttempts" return="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to get a report of all Application attempts
+ in the cluster from the <code>ApplicationHistoryServer</code>.
+ </p>
+
+ <p>
+ The <code>ApplicationHistoryServer</code> responds with a
+ {@link GetApplicationAttemptsRequest} which includes the
+ {@link ApplicationAttemptReport} for all the applications attempts of a
+ specified application attempt.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationAttemptReport(GetApplicationAttemptReportRequest)}.
+ </p>
+
+ @param request
+          request for reports on all application attempts of an application
+ @return reports on all application attempts of an application
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainerReport" return="org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to get a report of an Container from the
+ <code>ApplicationHistoryServer</code>.
+ </p>
+
+ <p>
+ The client, via {@link GetContainerReportRequest} provides the
+ {@link ContainerId} of the container.
+ </p>
+
+ <p>
+ In secure mode,the <code>ApplicationHistoryServer</code> verifies access to
+ the method before accepting the request.
+ </p>
+
+ <p>
+ The <code>ApplicationHistoryServer</code> responds with a
+ {@link GetContainerReportResponse} which includes the
+ {@link ContainerReport} for the container.
+ </p>
+
+ @param request
+          request for a container report
+ @return container report
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainers" return="org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to get a report of Containers for an
+ application attempt from the <code>ApplciationHistoryServer</code>.
+ </p>
+
+ <p>
+ The client, via {@link GetContainersRequest} provides the
+ {@link ApplicationAttemptId} of the application attempt.
+ </p>
+
+ <p>
+ In secure mode,the <code>ApplicationHistoryServer</code> verifies access to
+ the method before accepting the request.
+ </p>
+
+ <p>
+ The <code>ApplicationHistoryServer</code> responds with a
+ {@link GetContainersResponse} which includes a list of
+ {@link ContainerReport} for all the containers of a specific application
+ attempt.
+ </p>
+
+ @param request
+          request for a list of container reports of an application attempt.
+ @return reports on all containers of an application attempt
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getDelegationToken" return="org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to get delegation token, enabling the
+ containers to be able to talk to the service using those tokens.
+ </p>
+
+ <p>
+ The <code>ApplicationHistoryServer</code> responds with the delegation
+ token {@link Token} that can be used by the client to speak to this
+ service.
+ </p>
+
+ @param request
+          request to get a delegation token for the client.
+ @return delegation token that can be used to talk to this service
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>
+ The protocol between clients and the <code>ApplicationHistoryServer</code> to
+ get the information of completed applications etc.
+ </p>]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.api.ApplicationHistoryProtocol -->
+  <!-- start interface org.apache.hadoop.yarn.api.ApplicationMasterProtocol -->
+  <interface name="ApplicationMasterProtocol"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="registerApplicationMaster" return="org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by a new <code>ApplicationMaster</code> to register with
+ the <code>ResourceManager</code>.
+ </p>
+
+ <p>
+ The <code>ApplicationMaster</code> needs to provide details such as RPC
+ Port, HTTP tracking url etc. as specified in
+ {@link RegisterApplicationMasterRequest}.
+ </p>
+
+ <p>
+ The <code>ResourceManager</code> responds with critical details such as
+ maximum resource capabilities in the cluster as specified in
+ {@link RegisterApplicationMasterResponse}.
+ </p>
+
+ @param request
+          registration request
+ @return registration respose
+ @throws YarnException
+ @throws IOException
+ @throws InvalidApplicationMasterRequestException
+           The exception is thrown when an ApplicationMaster tries to
+           register more then once.
+ @see RegisterApplicationMasterRequest
+ @see RegisterApplicationMasterResponse]]>
+      </doc>
+    </method>
+    <method name="finishApplicationMaster" return="org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>The interface used by an <code>ApplicationMaster</code> to notify the
+ <code>ResourceManager</code> about its completion (success or failed).</p>
+
+ <p>The <code>ApplicationMaster</code> has to provide details such as
+ final state, diagnostics (in case of failures) etc. as specified in
+ {@link FinishApplicationMasterRequest}.</p>
+
+ <p>The <code>ResourceManager</code> responds with
+ {@link FinishApplicationMasterResponse}.</p>
+
+ @param request completion request
+ @return completion response
+ @throws YarnException
+ @throws IOException
+ @see FinishApplicationMasterRequest
+ @see FinishApplicationMasterResponse]]>
+      </doc>
+    </method>
+    <method name="allocate" return="org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The main interface between an <code>ApplicationMaster</code> and the
+ <code>ResourceManager</code>.
+ </p>
+
+ <p>
+ The <code>ApplicationMaster</code> uses this interface to provide a list of
+ {@link ResourceRequest} and returns unused {@link Container} allocated to
+ it via {@link AllocateRequest}. Optionally, the
+ <code>ApplicationMaster</code> can also <em>blacklist</em> resources which
+ it doesn't want to use.
+ </p>
+
+ <p>
+ This also doubles up as a <em>heartbeat</em> to let the
+ <code>ResourceManager</code> know that the <code>ApplicationMaster</code>
+ is alive. Thus, applications should periodically make this call to be kept
+ alive. The frequency depends on
+ {@link YarnConfiguration#RM_AM_EXPIRY_INTERVAL_MS} which defaults to
+ {@link YarnConfiguration#DEFAULT_RM_AM_EXPIRY_INTERVAL_MS}.
+ </p>
+
+ <p>
+ The <code>ResourceManager</code> responds with list of allocated
+ {@link Container}, status of completed containers and headroom information
+ for the application.
+ </p>
+
+ <p>
+ The <code>ApplicationMaster</code> can use the available headroom
+ (resources) to decide how to utilized allocated resources and make informed
+ decisions about future resource requests.
+ </p>
+
+ @param request
+          allocation request
+ @return allocation response
+ @throws YarnException
+ @throws IOException
+ @throws InvalidApplicationMasterRequestException
+           This exception is thrown when an ApplicationMaster calls allocate
+           without registering first.
+ @throws InvalidResourceBlacklistRequestException
+           This exception is thrown when an application provides an invalid
+           specification for blacklist of resources.
+ @throws InvalidResourceRequestException
+           This exception is thrown when a {@link ResourceRequest} is out of
+           the range of the configured lower and upper limits on the
+           resources.
+ @see AllocateRequest
+ @see AllocateResponse]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>The protocol between a live instance of <code>ApplicationMaster</code>
+ and the <code>ResourceManager</code>.</p>
+
+ <p>This is used by the <code>ApplicationMaster</code> to register/unregister
+ and to request and obtain resources in the cluster from the
+ <code>ResourceManager</code>.</p>]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.api.ApplicationMasterProtocol -->
+  <!-- start interface org.apache.hadoop.yarn.api.ContainerManagementProtocol -->
+  <interface name="ContainerManagementProtocol"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="startContainers" return="org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The <code>ApplicationMaster</code> provides a list of
+ {@link StartContainerRequest}s to a <code>NodeManager</code> to
+ <em>start</em> {@link Container}s allocated to it using this interface.
+ </p>
+
+ <p>
+ The <code>ApplicationMaster</code> has to provide details such as allocated
+ resource capability, security tokens (if enabled), command to be executed
+ to start the container, environment for the process, necessary
+ binaries/jar/shared-objects etc. via the {@link ContainerLaunchContext} in
+ the {@link StartContainerRequest}.
+ </p>
+
+ <p>
+ The <code>NodeManager</code> sends a response via
+ {@link StartContainersResponse} which includes a list of
+ {@link Container}s of successfully launched {@link Container}s, a
+ containerId-to-exception map for each failed {@link StartContainerRequest} in
+ which the exception indicates errors from per container and a
+ allServicesMetaData map between the names of auxiliary services and their
+ corresponding meta-data. Note: None-container-specific exceptions will
+ still be thrown by the API method itself.
+ </p>
+ <p>
+ The <code>ApplicationMaster</code> can use
+ {@link #getContainerStatuses(GetContainerStatusesRequest)} to get updated
+ statuses of the to-be-launched or launched containers.
+ </p>
+
+ @param request
+          request to start a list of containers
+ @return response including conatinerIds of all successfully launched
+         containers, a containerId-to-exception map for failed requests and
+         a allServicesMetaData map.
+ @throws YarnException
+ @throws IOException
+ @throws NMNotYetReadyException
+           This exception is thrown when NM starts from scratch but has not
+           yet connected with RM.]]>
+      </doc>
+    </method>
+    <method name="stopContainers" return="org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The <code>ApplicationMaster</code> requests a <code>NodeManager</code> to
+ <em>stop</em> a list of {@link Container}s allocated to it using this
+ interface.
+ </p>
+
+ <p>
+ The <code>ApplicationMaster</code> sends a {@link StopContainersRequest}
+ which includes the {@link ContainerId}s of the containers to be stopped.
+ </p>
+
+ <p>
+ The <code>NodeManager</code> sends a response via
+ {@link StopContainersResponse} which includes a list of {@link ContainerId}
+ s of successfully stopped containers, a containerId-to-exception map for
+ each failed request in which the exception indicates errors from per
+ container. Note: None-container-specific exceptions will still be thrown by
+ the API method itself. <code>ApplicationMaster</code> can use
+ {@link #getContainerStatuses(GetContainerStatusesRequest)} to get updated
+ statuses of the containers.
+ </p>
+
+ @param request
+          request to stop a list of containers
+ @return response which includes a list of containerIds of successfully
+         stopped containers, a containerId-to-exception map for failed
+         requests.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainerStatuses" return="org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The API used by the <code>ApplicationMaster</code> to request for current
+ statuses of <code>Container</code>s from the <code>NodeManager</code>.
+ </p>
+
+ <p>
+ The <code>ApplicationMaster</code> sends a
+ {@link GetContainerStatusesRequest} which includes the {@link ContainerId}s
+ of all containers whose statuses are needed.
+ </p>
+
+ <p>
+ The <code>NodeManager</code> responds with
+ {@link GetContainerStatusesResponse} which includes a list of
+ {@link ContainerStatus} of the successfully queried containers and a
+ containerId-to-exception map for each failed request in which the exception
+ indicates errors from per container. Note: None-container-specific
+ exceptions will still be thrown by the API method itself.
+ </p>
+
+ @param request
+          request to get <code>ContainerStatus</code>es of containers with
+          the specified <code>ContainerId</code>s
+ @return response containing the list of <code>ContainerStatus</code> of the
+         successfully queried containers and a containerId-to-exception map
+         for failed requests.
+
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>The protocol between an <code>ApplicationMaster</code> and a
+ <code>NodeManager</code> to start/stop containers and to get status
+ of running containers.</p>
+
+ <p>If security is enabled the <code>NodeManager</code> verifies that the
+ <code>ApplicationMaster</code> has truly been allocated the container
+ by the <code>ResourceManager</code> and also verifies all interactions such
+ as stopping the container or obtaining status information for the container.
+ </p>]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.api.ContainerManagementProtocol -->
+</package>
+<package name="org.apache.hadoop.yarn.api.protocolrecords">
+  <!-- start class org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest -->
+  <class name="AllocateRequest" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AllocateRequest"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="newInstance" return="org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="responseID" type="int"/>
+      <param name="appProgress" type="float"/>
+      <param name="resourceAsk" type="java.util.List"/>
+      <param name="containersToBeReleased" type="java.util.List"/>
+      <param name="resourceBlacklistRequest" type="org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest"/>
+    </method>
+    <method name="newInstance" return="org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="responseID" type="int"/>
+      <param name="appProgress" type="float"/>
+      <param name="resourceAsk" type="java.util.List"/>
+      <param name="containersToBeReleased" type="java.util.List"/>
+      <param name="resourceBlacklistRequest" type="org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest"/>
+      <param name="increaseRequests" type="java.util.List"/>
+    </method>
+    <method name="getResponseId" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <em>response id</em> used to track duplicate responses.
+ @return <em>response id</em>]]>
+      </doc>
+    </method>
+    <method name="setResponseId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="id" type="int"/>
+      <doc>
+      <![CDATA[Set the <em>response id</em> used to track duplicate responses.
+ @param id <em>response id</em>]]>
+      </doc>
+    </method>
+    <method name="getProgress" return="float"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <em>current progress</em> of application.
+ @return <em>current progress</em> of application]]>
+      </doc>
+    </method>
+    <method name="setProgress"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="progress" type="float"/>
+      <doc>
+      <![CDATA[Set the <em>current progress</em> of application
+ @param progress <em>current progress</em> of application]]>
+      </doc>
+    </method>
+    <method name="getAskList" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the list of <code>ResourceRequest</code> to update the
+ <code>ResourceManager</code> about the application's resource requirements.
+ @return the list of <code>ResourceRequest</code>
+ @see ResourceRequest]]>
+      </doc>
+    </method>
+    <method name="setAskList"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="resourceRequests" type="java.util.List"/>
+      <doc>
+      <![CDATA[Set list of <code>ResourceRequest</code> to update the
+ <code>ResourceManager</code> about the application's resource requirements.
+ @param resourceRequests list of <code>ResourceRequest</code> to update the
+                        <code>ResourceManager</code> about the application's
+                        resource requirements
+ @see ResourceRequest]]>
+      </doc>
+    </method>
+    <method name="getReleaseList" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the list of <code>ContainerId</code> of containers being
+ released by the <code>ApplicationMaster</code>.
+ @return list of <code>ContainerId</code> of containers being
+         released by the <code>ApplicationMaster</code>]]>
+      </doc>
+    </method>
+    <method name="setReleaseList"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="releaseContainers" type="java.util.List"/>
+      <doc>
+      <![CDATA[Set the list of <code>ContainerId</code> of containers being
+ released by the <code>ApplicationMaster</code>
+ @param releaseContainers list of <code>ContainerId</code> of
+                          containers being released by the
+                          <code>ApplicationMaster</code>]]>
+      </doc>
+    </method>
+    <method name="getResourceBlacklistRequest" return="org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <code>ResourceBlacklistRequest</code> being sent by the
+ <code>ApplicationMaster</code>.
+ @return the <code>ResourceBlacklistRequest</code> being sent by the
+         <code>ApplicationMaster</code>
+ @see ResourceBlacklistRequest]]>
+      </doc>
+    </method>
+    <method name="setResourceBlacklistRequest"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="resourceBlacklistRequest" type="org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest"/>
+      <doc>
+      <![CDATA[Set the <code>ResourceBlacklistRequest</code> to inform the
+ <code>ResourceManager</code> about the blacklist additions and removals
+ per the <code>ApplicationMaster</code>.
+
+ @param resourceBlacklistRequest the <code>ResourceBlacklistRequest</code>
+                         to inform the <code>ResourceManager</code> about
+                         the blacklist additions and removals
+                         per the <code>ApplicationMaster</code>
+ @see ResourceBlacklistRequest]]>
+      </doc>
+    </method>
+    <method name="getIncreaseRequests" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <code>ContainerResourceIncreaseRequest</code> being sent by the
+ <code>ApplicationMaster</code>]]>
+      </doc>
+    </method>
+    <method name="setIncreaseRequests"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="increaseRequests" type="java.util.List"/>
+      <doc>
+      <![CDATA[Set the <code>ContainerResourceIncreaseRequest</code> to inform the
+ <code>ResourceManager</code> about some container's resources need to be
+ increased]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>The core request sent by the <code>ApplicationMaster</code> to the
+ <code>ResourceManager</code> to obtain resources in the cluster.</p>
+
+ <p>The request includes:
+   <ul>
+     <li>A response id to track duplicate responses.</li>
+     <li>Progress information.</li>
+     <li>
+       A list of {@link ResourceRequest} to inform the
+       <code>ResourceManager</code> about the application's
+       resource requirements.
+     </li>
+     <li>
+       A list of unused {@link Container} which are being returned.
+     </li>
+   </ul>
+ </p>
+
+ @see ApplicationMasterProtocol#allocate(AllocateRequest)]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest -->
+  <!-- start class org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse -->
+  <class name="AllocateResponse" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AllocateResponse"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="newInstance" return="org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="responseId" type="int"/>
+      <param name="completedContainers" type="java.util.List"/>
+      <param name="allocatedContainers" type="java.util.List"/>
+      <param name="updatedNodes" type="java.util.List"/>
+      <param name="availResources" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <param name="command" type="org.apache.hadoop.yarn.api.records.AMCommand"/>
+      <param name="numClusterNodes" type="int"/>
+      <param name="preempt" type="org.apache.hadoop.yarn.api.records.PreemptionMessage"/>
+      <param name="nmTokens" type="java.util.List"/>
+    </method>
+    <method name="newInstance" return="org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="responseId" type="int"/>
+      <param name="completedContainers" type="java.util.List"/>
+      <param name="allocatedContainers" type="java.util.List"/>
+      <param name="updatedNodes" type="java.util.List"/>
+      <param name="availResources" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <param name="command" type="org.apache.hadoop.yarn.api.records.AMCommand"/>
+      <param name="numClusterNodes" type="int"/>
+      <param name="preempt" type="org.apache.hadoop.yarn.api.records.PreemptionMessage"/>
+      <param name="nmTokens" type="java.util.List"/>
+      <param name="increasedContainers" type="java.util.List"/>
+      <param name="decreasedContainers" type="java.util.List"/>
+    </method>
+    <method name="getAMCommand" return="org.apache.hadoop.yarn.api.records.AMCommand"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[If the <code>ResourceManager</code> needs the
+ <code>ApplicationMaster</code> to take some action then it will send an
+ AMCommand to the <code>ApplicationMaster</code>. See <code>AMCommand</code>
+ for details on commands and actions for them.
+ @return <code>AMCommand</code> if the <code>ApplicationMaster</code> should
+         take action, <code>null</code> otherwise
+ @see AMCommand]]>
+      </doc>
+    </method>
+    <method name="getResponseId" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <em>last response id</em>.
+ @return <em>last response id</em>]]>
+      </doc>
+    </method>
+    <method name="getAllocatedContainers" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the list of <em>newly allocated</em> <code>Container</code> by the
+ <code>ResourceManager</code>.
+ @return list of <em>newly allocated</em> <code>Container</code>]]>
+      </doc>
+    </method>
+    <method name="getAvailableResources" return="org.apache.hadoop.yarn.api.records.Resource"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <em>available headroom</em> for resources in the cluster for the
+ application.
+ @return limit of available headroom for resources in the cluster for the
+ application]]>
+      </doc>
+    </method>
+    <method name="getCompletedContainersStatuses" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the list of <em>completed containers' statuses</em>.
+ @return the list of <em>completed containers' statuses</em>]]>
+      </doc>
+    </method>
+    <method name="getUpdatedNodes" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the list of <em>updated <code>NodeReport</code>s</em>. Updates could
+ be changes in health, availability etc of the nodes.
+ @return The delta of updated nodes since the last response]]>
+      </doc>
+    </method>
+    <method name="getNumClusterNodes" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the number of hosts available on the cluster.
+ @return the available host count.]]>
+      </doc>
+    </method>
+    <method name="getPreemptionMessage" return="org.apache.hadoop.yarn.api.records.PreemptionMessage"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[<p>Get the description of containers owned by the AM, but requested back by
+ the cluster. Note that the RM may have an inconsistent view of the
+ resources owned by the AM. These messages are advisory, and the AM may
+ elect to ignore them.<p>
+
+ <p>The message is a snapshot of the resources the RM wants back from the AM.
+ While demand persists, the RM will repeat its request; applications should
+ not interpret each message as a request for <em>additional<em>
+ resources on top of previous messages. Resources requested consistently
+ over some duration may be forcibly killed by the RM.<p>
+
+ @return A specification of the resources to reclaim from this AM.]]>
+      </doc>
+    </method>
+    <method name="getNMTokens" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[<p>Get the list of NMTokens required for communicating with NM. New NMTokens
+ issued only if<p>
+ <p>1) AM is receiving first container on underlying NodeManager.<br>
+ OR<br>
+ 2) NMToken master key rolled over in ResourceManager and AM is getting new
+ container on the same underlying NodeManager.<p>
+ <p>AM will receive one NMToken per NM irrespective of the number of containers
+ issued on same NM. AM is expected to store these tokens until issued a
+ new token for the same NM.<p>]]>
+      </doc>
+    </method>
+    <method name="getIncreasedContainers" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the list of newly increased containers by <code>ResourceManager</code>]]>
+      </doc>
+    </method>
+    <method name="getDecreasedContainers" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the list of newly decreased containers by <code>NodeManager</code>]]>
+      </doc>
+    </method>
+    <method name="getAMRMToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[The AMRMToken that belong to this attempt
+
+ @return The AMRMToken that belong to this attempt]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>The response sent by the <code>ResourceManager</code> the
+ <code>ApplicationMaster</code> during resource negotiation.</p>
+
+ <p>The response, includes:
+   <ul>
+     <li>Response ID to track duplicate responses.</li>
+     <li>
+       An AMCommand sent by ResourceManager to let the <code>ApplicationMaster</code>
+       take some actions (resync, shutdown etc.).
+     <li>A list of newly allocated {@link Container}.</li>
+     <li>A list of completed {@link Container}s' statuses.</li>
+     <li>
+       The available headroom for resources in the cluster for the
+       application.
+     </li>
+     <li>A list of nodes whose status has been updated.</li>
+     <li>The number of available nodes in a cluster.</li>
+     <li>A description of resources requested back by the cluster</li>
+     <li>AMRMToken, if AMRMToken has been rolled over</li>
+   </ul>
+ </p>
+
+ @see ApplicationMasterProtocol#allocate(AllocateRequest)]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse -->
+  <!-- start class org.apache.hadoop.yarn.api.protocolrecords.ApplicationsRequestScope -->
+  <class name="ApplicationsRequestScope" extends="java.lang.Enum"
+    abstract="false"
+    static="false" final="true" visibility="public"
+    deprecated="not deprecated">
+    <method name="values" return="org.apache.hadoop.yarn.api.protocolrecords.ApplicationsRequestScope[]"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="valueOf" return="org.apache.hadoop.yarn.api.protocolrecords.ApplicationsRequestScope"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+    </method>
+    <doc>
+    <![CDATA[Enumeration that controls the scope of applications fetched]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.api.protocolrecords.ApplicationsRequestScope -->
+  <!-- start class org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest -->
+  <class name="FinishApplicationMasterRequest" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="FinishApplicationMasterRequest"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="newInstance" return="org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="finalAppStatus" type="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"/>
+      <param name="diagnostics" type="java.lang.String"/>
+      <param name="url" type="java.lang.String"/>
+    </method>
+    <method name="getFinalApplicationStatus" return="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get <em>final state</em> of the <code>ApplicationMaster</code>.
+ @return <em>final state</em> of the <code>ApplicationMaster</code>]]>
+      </doc>
+    </method>
+    <method name="setFinalApplicationStatus"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="finalState" type="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"/>
+      <doc>
+      <![CDATA[Set the <em>final state</em> of the <code>ApplicationMaster</code>
+ @param finalState <em>final state</em> of the <code>ApplicationMaster</code>]]>
+      </doc>
+    </method>
+    <method name="getDiagnostics" return="java.lang.String"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get <em>diagnostic information</em> on application failure.
+ @return <em>diagnostic information</em> on application failure]]>
+      </doc>
+    </method>
+    <method name="setDiagnostics"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="diagnostics" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Set <em>diagnostic information</em> on application failure.
+ @param diagnostics <em>diagnostic information</em> on application failure]]>
+      </doc>
+    </method>
+    <method name="getTrackingUrl" return="java.lang.String"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <em>tracking URL</em> for the <code>ApplicationMaster</code>.
+ This url if contains scheme then that will be used by resource manager
+ web application proxy otherwise it will default to http.
+ @return <em>tracking URL</em>for the <code>ApplicationMaster</code>]]>
+      </doc>
+    </method>
+    <method name="setTrackingUrl"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="url" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Set the <em>final tracking URL</em>for the <code>ApplicationMaster</code>.
+ This is the web-URL to which ResourceManager or web-application proxy will
+ redirect client/users once the application is finished and the
+ <code>ApplicationMaster</code> is gone.
+ <p>
+ If the passed url has a scheme then that will be used by the
+ ResourceManager and web-application proxy, otherwise the scheme will
+ default to http.
+ </p>
+ <p>
+ Empty, null, "N/A" strings are all valid besides a real URL. In case an url
+ isn't explicitly passed, it defaults to "N/A" on the ResourceManager.
+ <p>
+
+ @param url
+          <em>tracking URL</em>for the <code>ApplicationMaster</code>]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[<p>The finalization request sent by the <code>ApplicationMaster</code> to
+ inform the <code>ResourceManager</code> about its completion.</p>
+
+ <p>The final request includes details such:
+   <ul>
+     <li>Final state of the <code>ApplicationMaster</code></li>
+     <li>
+       Diagnostic information in

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[14/51] [abbrv] hadoop git commit: HDFS-10508. DFSInputStream should set thread's interrupt status after catching InterruptException from sleep. Contributed by Jing Zhao.

Posted by vv...@apache.org.
HDFS-10508. DFSInputStream should set thread's interrupt status after catching InterruptException from sleep. Contributed by Jing Zhao.


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

Branch: refs/heads/YARN-3926
Commit: 8ea9bbce2614e8eb499af73589f021ed1789e78f
Parents: 1500a0a
Author: Masatake Iwasaki <iw...@apache.org>
Authored: Thu Jun 9 14:52:29 2016 +0900
Committer: Masatake Iwasaki <iw...@apache.org>
Committed: Thu Jun 9 14:52:29 2016 +0900

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java      | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ea9bbce/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 7f32a56..6132f83 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -306,6 +306,7 @@ public class DFSInputStream extends FSInputStream
     try {
       Thread.sleep(waitTime);
     } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
       throw new InterruptedIOException(
           "Interrupted while getting the last block length.");
     }
@@ -417,6 +418,7 @@ public class DFSInputStream extends FSInputStream
         try {
           Thread.sleep(500); // delay between retries.
         } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
           throw new InterruptedIOException(
               "Interrupted while getting the length.");
         }
@@ -1063,6 +1065,7 @@ public class DFSInputStream extends FSInputStream
               " IOException, will wait for " + waitTime + " msec.");
           Thread.sleep((long)waitTime);
         } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
           throw new InterruptedIOException(
               "Interrupted while choosing DataNode for read.");
         }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[19/51] [abbrv] hadoop git commit: HADOOP-13175. Remove hadoop-ant from hadoop-tools. Contributed by Chris Douglas.

Posted by vv...@apache.org.
HADOOP-13175. Remove hadoop-ant from hadoop-tools. Contributed by Chris Douglas.


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

Branch: refs/heads/YARN-3926
Commit: 58c3486850c0503aecdeae8b67bb7e6bc42b4da8
Parents: 31ffaf7
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Jun 9 13:49:52 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Jun 9 13:49:52 2016 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml                          |   5 -
 hadoop-tools/hadoop-ant/pom.xml                 |  56 -----
 .../java/org/apache/hadoop/ant/DfsTask.java     | 220 -------------------
 .../ant/condition/DfsBaseConditional.java       |  68 ------
 .../apache/hadoop/ant/condition/DfsExists.java  |  24 --
 .../apache/hadoop/ant/condition/DfsIsDir.java   |  24 --
 .../apache/hadoop/ant/condition/DfsZeroLen.java |  24 --
 .../resources/org/apache/hadoop/ant/antlib.xml  |  29 ---
 hadoop-tools/hadoop-tools-dist/pom.xml          |   6 -
 hadoop-tools/pom.xml                            |   1 -
 10 files changed, 457 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 4c618a1..35166b1 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -375,11 +375,6 @@
         <artifactId>hadoop-extras</artifactId>
         <version>${project.version}</version>
       </dependency>
-      <dependency>
-        <groupId>org.apache.hadoop</groupId>
-        <artifactId>hadoop-ant</artifactId>
-        <version>${project.version}</version>
-      </dependency>
 
       <dependency>
         <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/hadoop-ant/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-ant/pom.xml b/hadoop-tools/hadoop-ant/pom.xml
deleted file mode 100644
index e0b038e..0000000
--- a/hadoop-tools/hadoop-ant/pom.xml
+++ /dev/null
@@ -1,56 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
-                      http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.hadoop</groupId>
-    <artifactId>hadoop-project</artifactId>
-    <version>3.0.0-alpha1-SNAPSHOT</version>
-    <relativePath>../../hadoop-project</relativePath>
-  </parent>
-  <groupId>org.apache.hadoop</groupId>
-  <artifactId>hadoop-ant</artifactId>
-  <version>3.0.0-alpha1-SNAPSHOT</version>
-  <description>Apache Hadoop Ant Tasks</description>
-  <name>Apache Hadoop Ant Tasks</name>
-  <packaging>jar</packaging>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.ant</groupId>
-      <artifactId>ant</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-annotations</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs-client</artifactId>
-      <scope>provided</scope>
-    </dependency>
-  </dependencies>
-
-</project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/DfsTask.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/DfsTask.java b/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/DfsTask.java
deleted file mode 100644
index 36119f5..0000000
--- a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/DfsTask.java
+++ /dev/null
@@ -1,220 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ant;
-
-import java.io.ByteArrayOutputStream;
-import java.io.OutputStream;
-import java.io.PrintStream;
-import java.io.UnsupportedEncodingException;
-import java.security.AccessController;
-import java.security.PrivilegedAction;
-import java.util.LinkedList;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FsShell;
-import org.apache.tools.ant.AntClassLoader;
-import org.apache.tools.ant.BuildException;
-import org.apache.tools.ant.Task;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * {@link org.apache.hadoop.fs.FsShell FsShell} wrapper for ant Task.
- */
-@InterfaceAudience.Private
-public class DfsTask extends Task {
-
-  static {
-    // adds the default resources
-    Configuration.addDefaultResource("hdfs-default.xml");
-    Configuration.addDefaultResource("hdfs-site.xml");
-  }
-
-  /**
-   * Default sink for {@link java.lang.System#out}
-   * and {@link java.lang.System#err}.
-   */
-  private static final OutputStream nullOut = new OutputStream() {
-      public void write(int b)    { /* ignore */ }
-      public String toString()    { return ""; }
-  };
-  private static final FsShell shell = new FsShell();
-
-  protected AntClassLoader confloader;
-  protected OutputStream out = nullOut;
-  protected OutputStream err = nullOut;
-
-  // set by ant
-  protected String cmd;
-  protected final LinkedList<String> argv = new LinkedList<String>();
-  protected String outprop;
-  protected String errprop;
-  protected boolean failonerror = true;
-
-  // saved ant context
-  private PrintStream antOut;
-  private PrintStream antErr;
-
-  /**
-   * Sets the command to run in {@link org.apache.hadoop.fs.FsShell FsShell}.
-   * @param cmd A valid command to FsShell, sans &quot;-&quot;.
-   */
-  public void setCmd(String cmd) {
-    this.cmd = "-" + cmd.trim();
-  }
-
-  /**
-   * Sets the argument list from a String of comma-separated values.
-   * @param args A String of comma-separated arguments to FsShell.
-   */
-  public void setArgs(String args) {
-    for (String s : args.trim().split("\\s*,\\s*"))
-      argv.add(s);
-  }
-
-  /**
-   * Sets the property into which System.out will be written.
-   * @param outprop The name of the property into which System.out is written.
-   * If the property is defined before this task is executed, it will not be updated.
-   */
-  public void setOut(String outprop) {
-    this.outprop = outprop;
-    out = new ByteArrayOutputStream();
-    if (outprop.equals(errprop))
-      err = out;
-  }
-
-  /**
-   * Sets the property into which System.err will be written. If this property
-   * has the same name as the property for System.out, the two will be interlaced.
-   * @param errprop The name of the property into which System.err is written.
-   * If the property is defined before this task is executed, it will not be updated.
-   */
-  public void setErr(String errprop) {
-    this.errprop = errprop;
-    err = (errprop.equals(outprop)) ? err = out : new ByteArrayOutputStream();
-  }
-
-  /**
-   * Sets the path for the parent-last ClassLoader, intended to be used for
-   * {@link org.apache.hadoop.conf.Configuration Configuration}.
-   * @param confpath The path to search for resources, classes, etc. before
-   * parent ClassLoaders.
-   */
-  public void setConf(String confpath) {
-    confloader = AccessController.doPrivileged(
-        new PrivilegedAction<AntClassLoader>() {
-          @Override
-          public AntClassLoader run() {
-            return new AntClassLoader(getClass().getClassLoader(), false);
-          }
-        });
-    confloader.setProject(getProject());
-    if (null != confpath)
-      confloader.addPathElement(confpath);
-  }
-
-  /**
-   * Sets a property controlling whether or not a
-   * {@link org.apache.tools.ant.BuildException BuildException} will be thrown
-   * if the command returns a value less than zero or throws an exception.
-   * @param failonerror If true, throw a BuildException on error.
-   */
-  public void setFailonerror(boolean failonerror) {
-    this.failonerror = failonerror;
-  }
-
-  /**
-   * Save the current values of System.out, System.err and configure output
-   * streams for FsShell.
-   */
-  protected void pushContext() {
-    antOut = System.out;
-    antErr = System.err;
-    try {
-      System.setOut(new PrintStream(out, false, "UTF-8"));
-      System.setErr(out == err ?
-          System.out : new PrintStream(err, false, "UTF-8"));
-    } catch (UnsupportedEncodingException ignored) {
-    }
-  }
-
-  /**
-   * Create the appropriate output properties with their respective output,
-   * restore System.out, System.err and release any resources from created
-   * ClassLoaders to aid garbage collection.
-   */
-  protected void popContext() {
-    // write output to property, if applicable
-    if (outprop != null && !System.out.checkError())
-      getProject().setNewProperty(outprop, out.toString());
-    if (out != err && errprop != null && !System.err.checkError())
-      getProject().setNewProperty(errprop, err.toString());
-
-    System.setErr(antErr);
-    System.setOut(antOut);
-    confloader.cleanup();
-    confloader.setParent(null);
-  }
-
-  // in case DfsTask is overridden
-  protected int postCmd(int exit_code) {
-    if ("-test".equals(cmd) && exit_code != 0)
-      outprop = null;
-    return exit_code;
-  }
-
-  /**
-   * Invoke {@link org.apache.hadoop.fs.FsShell#main} after a
-   * few cursory checks of the configuration.
-   */
-  public void execute() throws BuildException {
-    if (null == cmd)
-      throw new BuildException("Missing command (cmd) argument");
-    argv.add(0, cmd);
-
-    if (null == confloader) {
-      setConf(getProject().getProperty("hadoop.conf.dir"));
-    }
-
-    int exit_code = 0;
-    try {
-      pushContext();
-
-      Configuration conf = new Configuration();
-      conf.setClassLoader(confloader);
-      exit_code = ToolRunner.run(conf, shell,
-          argv.toArray(new String[argv.size()]));
-      exit_code = postCmd(exit_code);
-
-      if (0 > exit_code) {
-        StringBuilder msg = new StringBuilder();
-        for (String s : argv)
-          msg.append(s + " ");
-        msg.append("failed: " + exit_code);
-        throw new Exception(msg.toString());
-      }
-    } catch (Exception e) {
-      if (failonerror)
-          throw new BuildException(e);
-    } finally {
-      popContext();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsBaseConditional.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsBaseConditional.java b/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsBaseConditional.java
deleted file mode 100644
index 33cf52b..0000000
--- a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsBaseConditional.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ant.condition;
-
-import org.apache.tools.ant.taskdefs.condition.Condition;
-
-/**
- * This wrapper around {@link org.apache.hadoop.ant.DfsTask} implements the
- * Ant &gt;1.5
- * {@link org.apache.tools.ant.taskdefs.condition.Condition Condition}
- * interface for HDFS tests. So one can test conditions like this:
- * {@code
- *   <condition property="precond">
- *     <and>
- *       <hadoop:exists file="fileA" />
- *       <hadoop:exists file="fileB" />
- *       <hadoop:sizezero file="fileB" />
- *     </and>
- *   </condition>
- * }
- * This will define the property precond if fileA exists and fileB has zero
- * length.
- */
-public abstract class DfsBaseConditional extends org.apache.hadoop.ant.DfsTask
-                       implements Condition {
-
-  protected boolean result;
-  String file;
-
-  private void initArgs() {
-    setCmd("test");
-    setArgs("-"  +  getFlag() + "," + file);
-  }
-
-  public void setFile(String file) {
-    this.file = file;
-  }
-
-  protected abstract char getFlag();
-
-  protected int postCmd(int exit_code) {
-    exit_code = super.postCmd(exit_code);
-    result = exit_code == 0;
-    return exit_code;
-  }
-
-  public boolean eval() {
-    initArgs();
-    execute();
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsExists.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsExists.java b/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsExists.java
deleted file mode 100644
index 95e3868..0000000
--- a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsExists.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ant.condition;
-
-public class DfsExists extends DfsBaseConditional {
-  protected static final char flag = 'e';
-  protected char getFlag() { return flag; }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsIsDir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsIsDir.java b/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsIsDir.java
deleted file mode 100644
index b889615..0000000
--- a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsIsDir.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ant.condition;
-
-public class DfsIsDir extends DfsBaseConditional {
-  protected static final char flag = 'd';
-  protected char getFlag() { return flag; }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsZeroLen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsZeroLen.java b/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsZeroLen.java
deleted file mode 100644
index 70b4fbf..0000000
--- a/hadoop-tools/hadoop-ant/src/main/java/org/apache/hadoop/ant/condition/DfsZeroLen.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ant.condition;
-
-public class DfsZeroLen extends DfsBaseConditional {
-  protected static final char flag = 'z';
-  protected char getFlag() { return flag; }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/hadoop-ant/src/main/resources/org/apache/hadoop/ant/antlib.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-ant/src/main/resources/org/apache/hadoop/ant/antlib.xml b/hadoop-tools/hadoop-ant/src/main/resources/org/apache/hadoop/ant/antlib.xml
deleted file mode 100644
index 968b899..0000000
--- a/hadoop-tools/hadoop-ant/src/main/resources/org/apache/hadoop/ant/antlib.xml
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.0"?>
-
-<!--
-   Licensed to the Apache Software Foundation (ASF) under one or more
-   contributor license agreements.  See the NOTICE file distributed with
-   this work for additional information regarding copyright ownership.
-   The ASF licenses this file to You under the Apache License, Version 2.0
-   (the "License"); you may not use this file except in compliance with
-   the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
--->
-
-<antlib>
-  <taskdef name="hdfs"
-           classname="org.apache.hadoop.ant.DfsTask" />
-  <taskdef name="exists"
-           classname="org.apache.hadoop.ant.condition.DfsExists" />
-  <taskdef name="isdir"
-           classname="org.apache.hadoop.ant.condition.DfsIsDir" />
-  <taskdef name="sizezero"
-           classname="org.apache.hadoop.ant.condition.DfsZeroLen" />
-</antlib>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/hadoop-tools-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-tools-dist/pom.xml b/hadoop-tools/hadoop-tools-dist/pom.xml
index ef469fa..e1fbef1 100644
--- a/hadoop-tools/hadoop-tools-dist/pom.xml
+++ b/hadoop-tools/hadoop-tools-dist/pom.xml
@@ -105,12 +105,6 @@
       <artifactId>hadoop-sls</artifactId>
       <scope>compile</scope>
     </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-ant</artifactId>
-      <version>${project.version}</version>
-      <scope>compile</scope>
-    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58c34868/hadoop-tools/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/pom.xml b/hadoop-tools/pom.xml
index bb6d893..bd5f784 100644
--- a/hadoop-tools/pom.xml
+++ b/hadoop-tools/pom.xml
@@ -38,7 +38,6 @@
     <module>hadoop-rumen</module>
     <module>hadoop-gridmix</module>
     <module>hadoop-datajoin</module>
-    <module>hadoop-ant</module>
     <module>hadoop-tools-dist</module>
     <module>hadoop-extras</module>
     <module>hadoop-pipes</module>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[49/51] [abbrv] hadoop git commit: YARN-4715. Add support to read resource types from a config file. Contributed by Varun Vasudev.

Posted by vv...@apache.org.
YARN-4715. Add support to read resource types from a config file. Contributed by Varun Vasudev.


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

Branch: refs/heads/YARN-3926
Commit: 0bc39d3acf542f62a69643d49cf6e6595df90260
Parents: f203d5d
Author: Varun Vasudev <vv...@apache.org>
Authored: Fri Mar 11 15:03:15 2016 +0530
Committer: Varun Vasudev <vv...@apache.org>
Committed: Mon Jun 13 14:39:10 2016 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/api/records/Resource.java       |  24 +-
 .../yarn/api/records/ResourceInformation.java   |   8 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |  12 +
 .../exceptions/ResourceNotFoundException.java   |   2 +-
 .../hadoop-yarn/hadoop-yarn-common/pom.xml      |   8 +
 .../api/records/impl/pb/ResourcePBImpl.java     |  78 ++----
 .../resource/DominantResourceCalculator.java    |   5 +-
 .../yarn/util/resource/ResourceUtils.java       | 229 +++++++++++++++++
 .../hadoop/yarn/util/resource/Resources.java    |  18 +-
 .../src/main/resources/yarn-default.xml         |  10 +
 .../yarn/util/resource/TestResourceUtils.java   | 248 +++++++++++++++++++
 .../resource-types/resource-types-1.xml         |  18 ++
 .../resource-types/resource-types-2.xml         |  29 +++
 .../resource-types/resource-types-3.xml         |  24 ++
 .../resource-types/resource-types-4.xml         |  34 +++
 .../resource-types/resource-types-error-1.xml   |  29 +++
 .../resource-types/resource-types-error-2.xml   |  29 +++
 .../resource-types/resource-types-error-3.xml   |  29 +++
 .../resource-types/resource-types-error-4.xml   |  24 ++
 19 files changed, 765 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
index 7fca147..21d1aed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
@@ -66,15 +66,6 @@ public abstract class Resource implements Comparable<Resource> {
     return resource;
   }
 
-  @Public
-  @Stable
-  public static Resource newInstance(
-      Map<String, ResourceInformation> resources) {
-    Resource resource = Records.newRecord(Resource.class);
-    resource.setResources(resources);
-    return resource;
-  }
-
   /**
    * This method is DEPRECATED:
    * Use {@link Resource#getMemorySize()} instead
@@ -189,15 +180,6 @@ public abstract class Resource implements Comparable<Resource> {
   public abstract Long getResourceValue(String resource) throws YarnException;
 
   /**
-   * Set the resources to the map specified.
-   *
-   * @param resources Desired resources
-   */
-  @Public
-  @Evolving
-  public abstract void setResources(Map<String, ResourceInformation> resources);
-
-  /**
    * Set the ResourceInformation object for a particular resource.
    *
    * @param resource the resource for which the ResourceInformation is provided
@@ -231,8 +213,8 @@ public abstract class Resource implements Comparable<Resource> {
     result = prime * result + getVirtualCores();
     for (Map.Entry<String, ResourceInformation> entry : getResources()
         .entrySet()) {
-      if (entry.getKey().equals(ResourceInformation.MEMORY.getName()) || entry
-          .getKey().equals(ResourceInformation.VCORES.getName())) {
+      if (entry.getKey().equals(ResourceInformation.MEMORY_MB.getName())
+          || entry.getKey().equals(ResourceInformation.VCORES.getName())) {
         continue;
       }
       result = prime * result + entry.getValue().hashCode();
@@ -266,7 +248,7 @@ public abstract class Resource implements Comparable<Resource> {
         .append(getVirtualCores());
     for (Map.Entry<String, ResourceInformation> entry : getResources()
         .entrySet()) {
-      if (entry.getKey().equals(ResourceInformation.MEMORY.getName())
+      if (entry.getKey().equals(ResourceInformation.MEMORY_MB.getName())
           && entry.getValue().getUnits()
           .equals(ResourceInformation.MEMORY_MB.getUnits())) {
         continue;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
index 4e780c1..80e3192 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
@@ -32,11 +32,9 @@ public class ResourceInformation implements Comparable<ResourceInformation> {
   private ResourceTypes resourceType;
   private Long value;
 
-  private static final String MEMORY_URI = "yarn.io/memory";
-  private static final String VCORES_URI = "yarn.io/vcores";
+  private static final String MEMORY_URI = "memory-mb";
+  private static final String VCORES_URI = "vcores";
 
-  public static final ResourceInformation MEMORY =
-      ResourceInformation.newInstance(MEMORY_URI);
   public static final ResourceInformation MEMORY_MB =
       ResourceInformation.newInstance(MEMORY_URI, "M");
   public static final ResourceInformation VCORES =
@@ -77,7 +75,7 @@ public class ResourceInformation implements Comparable<ResourceInformation> {
   public void setUnits(String rUnits) {
     if (!UnitsConversionUtil.KNOWN_UNITS.contains(rUnits)) {
       throw new IllegalArgumentException(
-          "Unknown unit '" + units + "'. Known units are "
+          "Unknown unit '" + rUnits + "'. Known units are "
               + UnitsConversionUtil.KNOWN_UNITS);
     }
     this.units = rUnits;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/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 0beec62..a73d541 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
@@ -58,8 +58,13 @@ public class YarnConfiguration extends Configuration {
   public static final String CORE_SITE_CONFIGURATION_FILE = "core-site.xml";
 
   @Private
+  public static final String RESOURCE_TYPES_CONFIGURATION_FILE =
+      "resource-types.xml";
+
+  @Private
   public static final List<String> RM_CONFIGURATION_FILES =
       Collections.unmodifiableList(Arrays.asList(
+          RESOURCE_TYPES_CONFIGURATION_FILE,
           DR_CONFIGURATION_FILE,
           CS_CONFIGURATION_FILE,
           HADOOP_POLICY_CONFIGURATION_FILE,
@@ -89,6 +94,13 @@ public class YarnConfiguration extends Configuration {
 
   public static final String YARN_PREFIX = "yarn.";
 
+  /////////////////////////////
+  // Scheduler resource types configs
+  ////////////////////////////
+
+  public static final String RESOURCE_TYPES =
+      YarnConfiguration.YARN_PREFIX + "resource-types";
+
   /** Delay before deleting resource to ease debugging of NM issues */
   public static final String DEBUG_NM_DELETE_DELAY_SEC =
     YarnConfiguration.NM_PREFIX + "delete.debug-delay-sec";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ResourceNotFoundException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ResourceNotFoundException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ResourceNotFoundException.java
index 4277034..b5fece7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ResourceNotFoundException.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ResourceNotFoundException.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
-public class ResourceNotFoundException extends YarnException {
+public class ResourceNotFoundException extends YarnRuntimeException {
 
   private static final long serialVersionUID = 10081982L;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
index 17fc6e2..c96e270 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
@@ -199,6 +199,14 @@
         <filtering>true</filtering>
       </resource>
     </resources>
+    <testResources>
+      <testResource>
+        <directory>${project.basedir}/src/test/resources</directory>
+      </testResource>
+      <testResource>
+        <directory>${project.basedir}/src/test/resources/resource-types</directory>
+      </testResource>
+    </testResources>
     <plugins>
      <plugin>
         <groupId>org.apache.rat</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
index dc5c702..34e3607 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java
@@ -25,10 +25,11 @@ import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.exceptions.ResourceNotFoundException;
-import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProtoOrBuilder;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceInformationProto;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.apache.hadoop.yarn.util.UnitsConversionUtil;
 
 import java.util.*;
@@ -36,6 +37,7 @@ import java.util.*;
 @Private
 @Unstable
 public class ResourcePBImpl extends Resource {
+
   ResourceProto proto = ResourceProto.getDefaultInstance();
   ResourceProto.Builder builder = null;
   boolean viaProto = false;
@@ -75,41 +77,32 @@ public class ResourcePBImpl extends Resource {
 
   @Override
   public long getMemorySize() {
-    try {
-      ResourceInformation ri =
-          this.getResourceInformation(ResourceInformation.MEMORY.getName());
-      return (int) UnitsConversionUtil
-          .convert(ri.getUnits(), "M", ri.getValue()).longValue();
-    } catch (YarnException ye) {
-      // memory should always be present
-      initResourcesMap();
-      return 0;
-    }
+    // memory should always be present
+    initResourcesMap();
+    ResourceInformation ri =
+        this.getResourceInformation(ResourceInformation.MEMORY_MB.getName());
+    return UnitsConversionUtil.convert(ri.getUnits(), "M", ri.getValue());
   }
 
   @Override
   public void setMemory(long memory) {
     setResourceInformation(ResourceInformation.MEMORY_MB.getName(),
         ResourceInformation.newInstance(ResourceInformation.MEMORY_MB.getName(),
-            ResourceInformation.MEMORY_MB.getUnits(), (long) memory));
+            ResourceInformation.MEMORY_MB.getUnits(), memory));
 
   }
 
   @Override
   public int getVirtualCores() {
-    return (int) getVirtualCoresSize();
+    return (int) this.getVirtualCoresSize();
   }
 
+
   @Override
   public long getVirtualCoresSize() {
-    try {
-      return (int) this.getResourceValue(ResourceInformation.VCORES.getName())
-          .longValue();
-    } catch (YarnException ye) {
-      // vcores should always be present
-      initResourcesMap();
-      return 0;
-    }
+    // vcores should always be present
+    initResourcesMap();
+    return this.getResourceValue(ResourceInformation.VCORES.getName());
   }
 
   @Override
@@ -120,7 +113,7 @@ public class ResourcePBImpl extends Resource {
     } catch (ResourceNotFoundException re) {
       this.setResourceInformation(ResourceInformation.VCORES.getName(),
           ResourceInformation.newInstance(ResourceInformation.VCORES.getName(),
-              (long) vCores));
+              vCores));
     }
   }
 
@@ -149,21 +142,6 @@ public class ResourcePBImpl extends Resource {
   }
 
   @Override
-  public void setResources(Map<String, ResourceInformation> resources) {
-    maybeInitBuilder();
-    if (resources == null || resources.isEmpty()) {
-      builder.clearResourceValueMap();
-    } else {
-      for (Map.Entry<String, ResourceInformation> entry : resources.entrySet()) {
-        if (!entry.getKey().equals(entry.getValue().getName())) {
-          entry.getValue().setName(entry.getKey());
-        }
-      }
-    }
-    this.resources = resources;
-  }
-
-  @Override
   public void setResourceInformation(String resource,
       ResourceInformation resourceInformation) {
     maybeInitBuilder();
@@ -201,8 +179,7 @@ public class ResourcePBImpl extends Resource {
   }
 
   @Override
-  public ResourceInformation getResourceInformation(String resource)
-      throws YarnException {
+  public ResourceInformation getResourceInformation(String resource) {
     initResources();
     if (this.resources.containsKey(resource)) {
       return this.resources.get(resource);
@@ -211,7 +188,7 @@ public class ResourcePBImpl extends Resource {
   }
 
   @Override
-  public Long getResourceValue(String resource) throws YarnException {
+  public Long getResourceValue(String resource) {
     initResources();
     if (this.resources.containsKey(resource)) {
       return this.resources.get(resource).getValue();
@@ -222,18 +199,15 @@ public class ResourcePBImpl extends Resource {
   private void initResourcesMap() {
     if (resources == null) {
       resources = new HashMap<>();
-    }
-    ResourceInformation ri;
-    if (!resources.containsKey(ResourceInformation.MEMORY.getName())) {
-      ri = ResourceInformation
-          .newInstance(ResourceInformation.MEMORY_MB.getName(),
-              ResourceInformation.MEMORY_MB.getUnits());
-      this.resources.put(ResourceInformation.MEMORY.getName(), ri);
-    }
-    if (!resources.containsKey(ResourceInformation.VCORES.getName())) {
-      ri =
-          ResourceInformation.newInstance(ResourceInformation.VCORES.getName());
-      this.resources.put(ResourceInformation.VCORES.getName(), ri);
+      Map<String, ResourceInformation> types = ResourceUtils.getResourceTypes();
+      if (types == null) {
+        throw new YarnRuntimeException(
+            "Got null return value from ResourceUtils.getResourceTypes()");
+      }
+      for (Map.Entry<String, ResourceInformation> entry : types.entrySet()) {
+        resources.put(entry.getKey(),
+            ResourceInformation.newInstance(entry.getValue()));
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
index 1f3a8c2..c3748d2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.UnitsConversionUtil;
 
-import java.util.HashSet;
 import java.util.Set;
 
 /**
@@ -55,9 +54,7 @@ public class DominantResourceCalculator extends ResourceCalculator {
   private Set<String> resourceNames;
 
   public DominantResourceCalculator() {
-    resourceNames = new HashSet<>();
-    resourceNames.add(ResourceInformation.MEMORY.getName());
-    resourceNames.add(ResourceInformation.VCORES.getName());
+    resourceNames = ResourceUtils.getResourceTypes().keySet();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
new file mode 100644
index 0000000..0316f01
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
@@ -0,0 +1,229 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.util.resource;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
+import org.apache.hadoop.yarn.conf.ConfigurationProvider;
+import org.apache.hadoop.yarn.conf.ConfigurationProviderFactory;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Helper class to read the resource-types to be supported by the system.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ResourceUtils {
+
+  public static final String UNITS = ".units";
+  public static final String TYPE = ".type";
+
+  private static final Set<String> DISALLOWED_NAMES = new HashSet<>();
+  static {
+    DISALLOWED_NAMES.add("memory");
+    DISALLOWED_NAMES.add(ResourceInformation.MEMORY_MB.getName());
+    DISALLOWED_NAMES.add(ResourceInformation.VCORES.getName());
+  }
+
+  private static volatile Object lock;
+  private static Map<String, ResourceInformation> readOnlyResources;
+
+  static final Log LOG = LogFactory.getLog(ResourceUtils.class);
+
+  private ResourceUtils() {
+  }
+
+  private static void checkMandatatoryResources(
+      Map<String, ResourceInformation> resourceInformationMap)
+      throws YarnRuntimeException {
+    String memory = ResourceInformation.MEMORY_MB.getName();
+    String vcores = ResourceInformation.VCORES.getName();
+    if (resourceInformationMap.containsKey(memory)) {
+      ResourceInformation memInfo = resourceInformationMap.get(memory);
+      String memUnits = ResourceInformation.MEMORY_MB.getUnits();
+      ResourceTypes memType = ResourceInformation.MEMORY_MB.getResourceType();
+      if (!memInfo.getUnits().equals(memUnits) || !memInfo.getResourceType()
+          .equals(memType)) {
+        throw new YarnRuntimeException(
+            "Attempt to re-define mandatory resource 'memory-mb'. It can only"
+                + " be of type 'COUNTABLE' and have units 'M'.");
+      }
+    }
+
+    if (resourceInformationMap.containsKey(vcores)) {
+      ResourceInformation vcoreInfo = resourceInformationMap.get(vcores);
+      String vcoreUnits = ResourceInformation.VCORES.getUnits();
+      ResourceTypes vcoreType = ResourceInformation.VCORES.getResourceType();
+      if (!vcoreInfo.getUnits().equals(vcoreUnits) || !vcoreInfo
+          .getResourceType().equals(vcoreType)) {
+        throw new YarnRuntimeException(
+            "Attempt to re-define mandatory resource 'vcores'. It can only be"
+                + " of type 'COUNTABLE' and have units ''(no units).");
+      }
+    }
+  }
+
+  private static void addManadtoryResources(
+      Map<String, ResourceInformation> res) {
+    ResourceInformation ri;
+    if (!res.containsKey(ResourceInformation.MEMORY_MB.getName())) {
+      LOG.info("Adding resource type - name = " + ResourceInformation.MEMORY_MB
+          .getName() + ", units = " + ResourceInformation.MEMORY_MB.getUnits()
+          + ", type = " + ResourceTypes.COUNTABLE);
+      ri = ResourceInformation
+          .newInstance(ResourceInformation.MEMORY_MB.getName(),
+              ResourceInformation.MEMORY_MB.getUnits());
+      res.put(ResourceInformation.MEMORY_MB.getName(), ri);
+    }
+    if (!res.containsKey(ResourceInformation.VCORES.getName())) {
+      LOG.info("Adding resource type - name = " + ResourceInformation.VCORES
+          .getName() + ", units = , type = " + ResourceTypes.COUNTABLE);
+      ri =
+          ResourceInformation.newInstance(ResourceInformation.VCORES.getName());
+      res.put(ResourceInformation.VCORES.getName(), ri);
+    }
+  }
+
+  @VisibleForTesting
+  static void initializeResourcesMap(Configuration conf,
+      Map<String, ResourceInformation> resourceInformationMap) {
+
+    String[] resourceNames = conf.getStrings(YarnConfiguration.RESOURCE_TYPES);
+    if (resourceNames != null && resourceNames.length != 0) {
+      for (String resourceName : resourceNames) {
+        String resourceUnits = conf.get(
+            YarnConfiguration.RESOURCE_TYPES + "." + resourceName + UNITS, "");
+        String resourceTypeName = conf.get(
+            YarnConfiguration.RESOURCE_TYPES + "." + resourceName + TYPE,
+            ResourceTypes.COUNTABLE.toString());
+        if (resourceName == null || resourceName.isEmpty()
+            || resourceUnits == null || resourceTypeName == null) {
+          throw new YarnRuntimeException(
+              "Incomplete configuration for resource type '" + resourceName
+                  + "'. One of name, units or type is configured incorrectly.");
+        }
+        if (DISALLOWED_NAMES.contains(resourceName)) {
+          throw new YarnRuntimeException(
+              "Resource type cannot be named '" + resourceName
+                  + "'. That name is disallowed.");
+        }
+        ResourceTypes resourceType = ResourceTypes.valueOf(resourceTypeName);
+        LOG.info("Adding resource type - name = " + resourceName + ", units = "
+            + resourceUnits + ", type = " + resourceTypeName);
+        if (resourceInformationMap.containsKey(resourceName)) {
+          throw new YarnRuntimeException(
+              "Error in config, key '" + resourceName + "' specified twice");
+        }
+        resourceInformationMap.put(resourceName, ResourceInformation
+            .newInstance(resourceName, resourceUnits, 0L, resourceType));
+      }
+    }
+    checkMandatatoryResources(resourceInformationMap);
+    addManadtoryResources(resourceInformationMap);
+    readOnlyResources = Collections.unmodifiableMap(resourceInformationMap);
+  }
+
+  /**
+   * Get the resource types to be supported by the system.
+   * @return A map of the resource name to a ResouceInformation object
+   *         which contains details such as the unit.
+   */
+  public static Map<String, ResourceInformation> getResourceTypes() {
+    return getResourceTypes(null,
+        YarnConfiguration.RESOURCE_TYPES_CONFIGURATION_FILE);
+  }
+
+  private static Map<String, ResourceInformation> getResourceTypes(
+      Configuration conf, String resourceFile) {
+    if (lock == null) {
+      synchronized (ResourceUtils.class) {
+        if (lock == null) {
+          synchronized (ResourceUtils.class) {
+            lock = new Object();
+            Map<String, ResourceInformation> resources = new HashMap<>();
+            if (conf == null) {
+              conf = new YarnConfiguration();
+            }
+            try {
+              InputStream ris = getConfInputStream(resourceFile, conf);
+              LOG.debug("Found " + resourceFile + ", adding to configuration");
+              conf.addResource(ris);
+              initializeResourcesMap(conf, resources);
+              return resources;
+            } catch (FileNotFoundException fe) {
+              LOG.info("Unable to find '" + resourceFile
+                  + "'. Falling back to memory and vcores as resources", fe);
+              initializeResourcesMap(conf, resources);
+            } catch (IOException ie) {
+              LOG.fatal(
+                  "Exception trying to read resource types configuration '"
+                      + resourceFile + "'.", ie);
+              throw new YarnRuntimeException(ie);
+            } catch (YarnException ye) {
+              LOG.fatal(
+                  "YARN Exception trying to read resource types configuration '"
+                      + resourceFile + "'.", ye);
+              throw new YarnRuntimeException(ye);
+            }
+          }
+        }
+      }
+    }
+    return readOnlyResources;
+  }
+
+  static InputStream getConfInputStream(String resourceFile, Configuration conf)
+      throws IOException, YarnException {
+
+    ConfigurationProvider provider =
+        ConfigurationProviderFactory.getConfigurationProvider(conf);
+    InputStream ris = provider.getConfigurationInputStream(conf, resourceFile);
+    if (ris == null) {
+      if (conf.getResource(resourceFile) == null) {
+        throw new FileNotFoundException("Unable to find " + resourceFile);
+      }
+      throw new IOException(
+          "Unable to open resource types file '" + resourceFile
+              + "'. Using provider " + provider);
+    }
+    return ris;
+  }
+
+  @VisibleForTesting
+  static void resetResourceTypes() {
+    lock = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
index c9ff64f..dd08675 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
@@ -119,11 +119,6 @@ public class Resources {
     }
 
     @Override
-    public void setResources(Map<String, ResourceInformation> resources) {
-      throw new RuntimeException(name + " cannot be modified!");
-    }
-
-    @Override
     public void setResourceInformation(String resource,
         ResourceInformation resourceInformation)
         throws ResourceNotFoundException {
@@ -142,11 +137,14 @@ public class Resources {
       // needs to be Integer.MAX_VALUE
       int max = resourceValue > Integer.MAX_VALUE ? Integer.MAX_VALUE :
           resourceValue.intValue();
-      tmp.put(ResourceInformation.MEMORY.getName(), ResourceInformation
-          .newInstance(ResourceInformation.MEMORY.getName(),
-              ResourceInformation.MEMORY_MB.getUnits(), (long) max));
-      tmp.put(ResourceInformation.VCORES.getName(), ResourceInformation
-          .newInstance(ResourceInformation.VCORES.getName(), (long) max));
+      Map<String, ResourceInformation> types = ResourceUtils.getResourceTypes();
+      if (types != null) {
+        for (Map.Entry<String, ResourceInformation> entry : types.entrySet()) {
+          tmp.put(entry.getKey(),
+              ResourceInformation.newInstance(entry.getValue()));
+          tmp.get(entry.getKey()).setValue((long) max);
+        }
+      }
       return tmp;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/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 eabb679..e609ade 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
@@ -2800,4 +2800,14 @@
     <name>yarn.resourcemanager.node-removal-untracked.timeout-ms</name>
     <value>60000</value>
   </property>
+
+  <!-- resource types configuration -->
+  <property>
+    <name>yarn.resource-types</name>
+    <value></value>
+    <description>
+    The resource types to be used for scheduling. Use resource-types.xml
+    to specify details about the individual resource types.
+    </description>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java
new file mode 100644
index 0000000..606e89c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceUtils.java
@@ -0,0 +1,248 @@
+/**
+ * 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.util.resource;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.util.HashMap;
+import java.util.Map;
+
+public class TestResourceUtils {
+
+  static class ResourceFileInformation {
+    String filename;
+    int resourceCount;
+    Map<String, String> resourceNameUnitsMap;
+
+    public ResourceFileInformation(String name, int count) {
+      filename = name;
+      resourceCount = count;
+      resourceNameUnitsMap = new HashMap<>();
+    }
+  }
+
+  @Before
+  public void setup() {
+    ResourceUtils.resetResourceTypes();
+  }
+
+  private void testMemoryAndVcores(Map<String, ResourceInformation> res) {
+    String memory = ResourceInformation.MEMORY_MB.getName();
+    String vcores = ResourceInformation.VCORES.getName();
+    Assert.assertTrue("Resource 'memory' missing", res.containsKey(memory));
+    Assert.assertEquals("'memory' units incorrect",
+        ResourceInformation.MEMORY_MB.getUnits(), res.get(memory).getUnits());
+    Assert.assertEquals("'memory' types incorrect",
+        ResourceInformation.MEMORY_MB.getResourceType(),
+        res.get(memory).getResourceType());
+    Assert.assertTrue("Resource 'vcores' missing", res.containsKey(vcores));
+    Assert.assertEquals("'vcores' units incorrect",
+        ResourceInformation.VCORES.getUnits(), res.get(vcores).getUnits());
+    Assert.assertEquals("'vcores' type incorrect",
+        ResourceInformation.VCORES.getResourceType(),
+        res.get(vcores).getResourceType());
+  }
+
+  @Test
+  public void testGetResourceTypes() throws Exception {
+
+    Map<String, ResourceInformation> res = ResourceUtils.getResourceTypes();
+    Assert.assertEquals(2, res.size());
+    testMemoryAndVcores(res);
+  }
+
+  @Test
+  public void testGetResourceTypesConfigs() throws Exception {
+
+    Configuration conf = new YarnConfiguration();
+
+    ResourceFileInformation testFile1 =
+        new ResourceFileInformation("resource-types-1.xml", 2);
+    ResourceFileInformation testFile2 =
+        new ResourceFileInformation("resource-types-2.xml", 3);
+    testFile2.resourceNameUnitsMap.put("resource1", "G");
+    ResourceFileInformation testFile3 =
+        new ResourceFileInformation("resource-types-3.xml", 3);
+    testFile3.resourceNameUnitsMap.put("resource2", "");
+    ResourceFileInformation testFile4 =
+        new ResourceFileInformation("resource-types-4.xml", 4);
+    testFile4.resourceNameUnitsMap.put("resource1", "G");
+    testFile4.resourceNameUnitsMap.put("resource2", "m");
+
+    ResourceFileInformation[] tests =
+        { testFile1, testFile2, testFile3, testFile4 };
+    Map<String, ResourceInformation> res;
+    for (ResourceFileInformation testInformation : tests) {
+      ResourceUtils.resetResourceTypes();
+      File source = new File(
+          conf.getClassLoader().getResource(testInformation.filename)
+              .getFile());
+      File dest = new File(source.getParent(), "resource-types.xml");
+      FileUtils.copyFile(source, dest);
+      res = ResourceUtils.getResourceTypes();
+      testMemoryAndVcores(res);
+      Assert.assertEquals(testInformation.resourceCount, res.size());
+      for (Map.Entry<String, String> entry : testInformation.resourceNameUnitsMap
+          .entrySet()) {
+        String resourceName = entry.getKey();
+        Assert.assertTrue("Missing key " + resourceName,
+            res.containsKey(resourceName));
+        Assert.assertEquals(entry.getValue(), res.get(resourceName).getUnits());
+      }
+      dest.delete();
+    }
+  }
+
+  @Test
+  public void testGetResourceTypesConfigErrors() throws Exception {
+    Configuration conf = new YarnConfiguration();
+
+    String[] resourceFiles =
+        { "resource-types-error-1.xml", "resource-types-error-2.xml",
+            "resource-types-error-3.xml", "resource-types-error-4.xml" };
+    for (String resourceFile : resourceFiles) {
+      ResourceUtils.resetResourceTypes();
+      File dest = null;
+      try {
+        File source =
+            new File(conf.getClassLoader().getResource(resourceFile).getFile());
+        dest = new File(source.getParent(), "resource-types.xml");
+        FileUtils.copyFile(source, dest);
+        ResourceUtils.getResourceTypes();
+        Assert.fail("Expected error with file " + resourceFile);
+      } catch (NullPointerException ne) {
+        throw ne;
+      } catch (Exception e) {
+        if (dest != null) {
+          dest.delete();
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testInitializeResourcesMap() throws Exception {
+    String[] empty = { "", "" };
+    String[] res1 = { "resource1", "m" };
+    String[] res2 = { "resource2", "G" };
+    String[][] test1 = { empty };
+    String[][] test2 = { res1 };
+    String[][] test3 = { res2 };
+    String[][] test4 = { res1, res2 };
+
+    String[][][] allTests = { test1, test2, test3, test4 };
+
+    for (String[][] test : allTests) {
+
+      Configuration conf = new YarnConfiguration();
+      String resSt = "";
+      for (String[] resources : test) {
+        resSt += (resources[0] + ",");
+      }
+      resSt = resSt.substring(0, resSt.length() - 1);
+      conf.set(YarnConfiguration.RESOURCE_TYPES, resSt);
+      for (String[] resources : test) {
+        String name =
+            YarnConfiguration.RESOURCE_TYPES + "." + resources[0] + ".units";
+        conf.set(name, resources[1]);
+      }
+      Map<String, ResourceInformation> ret = new HashMap<>();
+      ResourceUtils.initializeResourcesMap(conf, ret);
+      // for test1, 4 - length will be 1, 4
+      // for the others, len will be 3
+      int len = 3;
+      if (test == test1) {
+        len = 2;
+      } else if (test == test4) {
+        len = 4;
+      }
+
+      Assert.assertEquals(len, ret.size());
+      for (String[] resources : test) {
+        if (resources[0].length() == 0) {
+          continue;
+        }
+        Assert.assertTrue(ret.containsKey(resources[0]));
+        ResourceInformation resInfo = ret.get(resources[0]);
+        Assert.assertEquals(resources[1], resInfo.getUnits());
+        Assert.assertEquals(ResourceTypes.COUNTABLE, resInfo.getResourceType());
+      }
+      // we must always have memory and vcores with their fixed units
+      Assert.assertTrue(ret.containsKey("memory-mb"));
+      ResourceInformation memInfo = ret.get("memory-mb");
+      Assert.assertEquals("M", memInfo.getUnits());
+      Assert.assertEquals(ResourceTypes.COUNTABLE, memInfo.getResourceType());
+      Assert.assertTrue(ret.containsKey("vcores"));
+      ResourceInformation vcoresInfo = ret.get("vcores");
+      Assert.assertEquals("", vcoresInfo.getUnits());
+      Assert
+          .assertEquals(ResourceTypes.COUNTABLE, vcoresInfo.getResourceType());
+    }
+  }
+
+  @Test
+  public void testInitializeResourcesMapErrors() throws Exception {
+
+    String[] mem1 = { "memory-mb", "" };
+    String[] vcores1 = { "vcores", "M" };
+
+    String[] mem2 = { "memory-mb", "m" };
+    String[] vcores2 = { "vcores", "G" };
+
+    String[] mem3 = { "memory", "" };
+
+    String[][] test1 = { mem1, vcores1 };
+    String[][] test2 = { mem2, vcores2 };
+    String[][] test3 = { mem3 };
+
+    String[][][] allTests = { test1, test2, test3 };
+
+    for (String[][] test : allTests) {
+
+      Configuration conf = new YarnConfiguration();
+      String resSt = "";
+      for (String[] resources : test) {
+        resSt += (resources[0] + ",");
+      }
+      resSt = resSt.substring(0, resSt.length() - 1);
+      conf.set(YarnConfiguration.RESOURCE_TYPES, resSt);
+      for (String[] resources : test) {
+        String name =
+            YarnConfiguration.RESOURCE_TYPES + "." + resources[0] + ".units";
+        conf.set(name, resources[1]);
+      }
+      Map<String, ResourceInformation> ret = new HashMap<>();
+      try {
+        ResourceUtils.initializeResourcesMap(conf, ret);
+        Assert.fail("resource map initialization should fail");
+      } catch (Exception e) {
+        // do nothing
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-1.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-1.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-1.xml
new file mode 100644
index 0000000..3ec106d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-1.xml
@@ -0,0 +1,18 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License. See accompanying LICENSE file.
+-->
+
+<configuration>
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-2.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-2.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-2.xml
new file mode 100644
index 0000000..6e5885e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-2.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License. See accompanying LICENSE file.
+-->
+
+<configuration>
+
+ <property>
+   <name>yarn.resource-types</name>
+   <value>resource1</value>
+ </property>
+
+ <property>
+   <name>yarn.resource-types.resource1.units</name>
+   <value>G</value>
+ </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-3.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-3.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-3.xml
new file mode 100644
index 0000000..8fd6fef
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-3.xml
@@ -0,0 +1,24 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License. See accompanying LICENSE file.
+-->
+
+<configuration>
+
+ <property>
+   <name>yarn.resource-types</name>
+   <value>resource2</value>
+ </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-4.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-4.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-4.xml
new file mode 100644
index 0000000..c84316a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-4.xml
@@ -0,0 +1,34 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License. See accompanying LICENSE file.
+-->
+
+<configuration>
+
+ <property>
+   <name>yarn.resource-types</name>
+   <value>resource1,resource2</value>
+ </property>
+
+ <property>
+   <name>yarn.resource-types.resource1.units</name>
+   <value>G</value>
+ </property>
+
+ <property>
+   <name>yarn.resource-types.resource2.units</name>
+   <value>m</value>
+ </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-error-1.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-error-1.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-error-1.xml
new file mode 100644
index 0000000..d1942f2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-error-1.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License. See accompanying LICENSE file.
+-->
+
+<configuration>
+
+ <property>
+   <name>yarn.resource-types</name>
+   <value>memory-mb,resource1</value>
+ </property>
+
+ <property>
+   <name>yarn.resource-types.resource1.calculator-units</name>
+   <value>G</value>
+ </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-error-2.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-error-2.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-error-2.xml
new file mode 100644
index 0000000..ca428eb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-error-2.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License. See accompanying LICENSE file.
+-->
+
+<configuration>
+
+ <property>
+   <name>yarn.resource-types</name>
+   <value>vcores,resource1</value>
+ </property>
+
+ <property>
+   <name>yarn.resource-types.resource1.calculator-units</name>
+   <value>G</value>
+ </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-error-3.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-error-3.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-error-3.xml
new file mode 100644
index 0000000..08b8a6d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-error-3.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License. See accompanying LICENSE file.
+-->
+
+<configuration>
+
+ <property>
+   <name>yarn.resource-types</name>
+   <value>vcores,resource1</value>
+ </property>
+
+ <property>
+   <name>yarn.resource-types.resource1.calculator-units</name>
+   <value>A</value>
+ </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bc39d3a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-error-4.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-error-4.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-error-4.xml
new file mode 100644
index 0000000..c8eb766
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/resources/resource-types/resource-types-error-4.xml
@@ -0,0 +1,24 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License. See accompanying LICENSE file.
+-->
+
+<configuration>
+
+ <property>
+   <name>yarn.resource-types</name>
+   <value>memory,resource1</value>
+ </property>
+
+</configuration>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[15/51] [abbrv] hadoop git commit: HADOOP-13220. Follow on fixups after upgraded mini-kdc using Kerby. Contributed by Jiajia Li

Posted by vv...@apache.org.
HADOOP-13220. Follow on fixups after upgraded mini-kdc using Kerby. Contributed by Jiajia Li


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

Branch: refs/heads/YARN-3926
Commit: 723432b3387fe69e6bf2b56d2ea1a7b1bda16b97
Parents: 76f0800
Author: Kai Zheng <ka...@intel.com>
Authored: Thu Jun 9 15:56:12 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Thu Jun 9 15:56:12 2016 +0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-auth/pom.xml       |  1 -
 hadoop-common-project/hadoop-common/pom.xml     |  1 -
 .../dev-support/findbugsExcludeFile.xml         | 28 ++++++++++++++++++++
 hadoop-common-project/hadoop-minikdc/pom.xml    | 14 +++++++++-
 hadoop-project/pom.xml                          |  6 +++++
 5 files changed, 47 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/723432b3/hadoop-common-project/hadoop-auth/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/pom.xml b/hadoop-common-project/hadoop-auth/pom.xml
index 27e4547..93dceb3 100644
--- a/hadoop-common-project/hadoop-auth/pom.xml
+++ b/hadoop-common-project/hadoop-auth/pom.xml
@@ -134,7 +134,6 @@
     <dependency>
       <groupId>org.apache.kerby</groupId>
       <artifactId>kerb-simplekdc</artifactId>
-      <version>1.0.0-RC2</version>
     </dependency>
   </dependencies>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/723432b3/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index 8bf052c..059986f 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -298,7 +298,6 @@
     <dependency>
       <groupId>org.apache.kerby</groupId>
       <artifactId>kerb-simplekdc</artifactId>
-      <version>1.0.0-RC2</version>
     </dependency>
   </dependencies>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/723432b3/hadoop-common-project/hadoop-minikdc/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-minikdc/dev-support/findbugsExcludeFile.xml b/hadoop-common-project/hadoop-minikdc/dev-support/findbugsExcludeFile.xml
new file mode 100644
index 0000000..9a1c4a7
--- /dev/null
+++ b/hadoop-common-project/hadoop-minikdc/dev-support/findbugsExcludeFile.xml
@@ -0,0 +1,28 @@
+<!--
+   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.
+-->
+<FindBugsFilter>
+  <!--
+    Caller is not supposed to modify returned values even though there's nothing
+    stopping them; we do this for performance reasons.
+  -->
+  <Match>
+    <Class name="org.apache.hadoop.minikdc.MiniKdc" />
+    <Method name="stop" />
+    <Bug pattern="SWL_SLEEP_WITH_LOCK_HELD" />
+  </Match>
+
+</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/723432b3/hadoop-common-project/hadoop-minikdc/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-minikdc/pom.xml b/hadoop-common-project/hadoop-minikdc/pom.xml
index 2e22ad0..3075cad 100644
--- a/hadoop-common-project/hadoop-minikdc/pom.xml
+++ b/hadoop-common-project/hadoop-minikdc/pom.xml
@@ -38,7 +38,6 @@
     <dependency>
       <groupId>org.apache.kerby</groupId>
       <artifactId>kerb-simplekdc</artifactId>
-      <version>1.0.0-RC2</version>
     </dependency>
     <dependency>
       <groupId>org.slf4j</groupId>
@@ -51,4 +50,17 @@
       <scope>compile</scope>
     </dependency>
   </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <excludeFilterFile>${basedir}/dev-support/findbugsExcludeFile.xml
+          </excludeFilterFile>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/723432b3/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index aa47f6c..2b6b162 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -1011,6 +1011,12 @@
             <version>1.3.0</version>
         </dependency>
 
+        <dependency>
+          <groupId>org.apache.kerby</groupId>
+          <artifactId>kerb-simplekdc</artifactId>
+          <version>1.0.0-RC2</version>
+        </dependency>
+
     </dependencies>
   </dependencyManagement>
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[23/51] [abbrv] hadoop git commit: MAPREDUCE-6741. Refactor UncompressedSplitLineReader.fillBuffer(). Contributed by Daniel Templeton.

Posted by vv...@apache.org.
MAPREDUCE-6741. Refactor UncompressedSplitLineReader.fillBuffer(). Contributed by Daniel Templeton.


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

Branch: refs/heads/YARN-3926
Commit: 0b7b8a377611b2a3041a2995504a437c36dfa6e6
Parents: 9581fb7
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Jun 10 19:15:36 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Jun 10 19:15:36 2016 +0900

----------------------------------------------------------------------
 .../mapreduce/lib/input/UncompressedSplitLineReader.java     | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b7b8a37/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/UncompressedSplitLineReader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/UncompressedSplitLineReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/UncompressedSplitLineReader.java
index bda0218..c2b005b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/UncompressedSplitLineReader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/UncompressedSplitLineReader.java
@@ -53,10 +53,10 @@ public class UncompressedSplitLineReader extends SplitLineReader {
       throws IOException {
     int maxBytesToRead = buffer.length;
     if (totalBytesRead < splitLength) {
-      long leftBytesForSplit = splitLength - totalBytesRead;
-      // check if leftBytesForSplit exceed Integer.MAX_VALUE
-      if (leftBytesForSplit <= Integer.MAX_VALUE) {
-        maxBytesToRead = Math.min(maxBytesToRead, (int)leftBytesForSplit);
+      long bytesLeftInSplit = splitLength - totalBytesRead;
+
+      if (bytesLeftInSplit < maxBytesToRead) {
+        maxBytesToRead = (int)bytesLeftInSplit;
       }
     }
     int bytesRead = in.read(buffer, 0, maxBytesToRead);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[08/51] [abbrv] hadoop git commit: YARN-5204. Properly report status of killed/stopped queued containers. (Konstantinos Karanasos via asuresh)

Posted by vv...@apache.org.
YARN-5204. Properly report status of killed/stopped queued containers. (Konstantinos Karanasos via asuresh)


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

Branch: refs/heads/YARN-3926
Commit: 3344ba70e027c929e07bad5e6877c796d41181e9
Parents: 8c8a377
Author: Arun Suresh <as...@apache.org>
Authored: Wed Jun 8 08:31:32 2016 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Wed Jun 8 08:31:32 2016 -0700

----------------------------------------------------------------------
 .../queuing/QueuingContainerManagerImpl.java    |  15 ++-
 .../queuing/TestQueuingContainerManager.java    | 129 +++++++++++++++----
 2 files changed, 115 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3344ba70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java
index a1e3bdb..38b1b07 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/QueuingContainerManagerImpl.java
@@ -175,8 +175,9 @@ public class QueuingContainerManagerImpl extends ContainerManagerImpl {
       }
 
       nodeStatusUpdater.sendOutofBandHeartBeat();
+    } else {
+      super.stopContainerInternal(containerID);
     }
-    super.stopContainerInternal(containerID);
   }
 
   /**
@@ -456,6 +457,18 @@ public class QueuingContainerManagerImpl extends ContainerManagerImpl {
             ContainerExitStatus.INVALID, this.context.getQueuingContext()
                 .getQueuedContainers().get(containerID).getResource(),
             executionType);
+      } else {
+        // Check if part of the stopped/killed queued containers.
+        for (ContainerTokenIdentifier cTokenId : this.context
+            .getQueuingContext().getKilledQueuedContainers().keySet()) {
+          if (cTokenId.getContainerID().equals(containerID)) {
+            return BuilderUtils.newContainerStatus(containerID,
+                org.apache.hadoop.yarn.api.records.ContainerState.COMPLETE,
+                this.context.getQueuingContext().getKilledQueuedContainers()
+                    .get(cTokenId), ContainerExitStatus.ABORTED, cTokenId
+                        .getResource(), cTokenId.getExecutionType());
+          }
+        }
       }
     }
     return super.getContainerStatusInternal(containerID, nmTokenIdentifier);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3344ba70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java
index 4d44d8d..caebef7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/queuing/TestQueuingContainerManager.java
@@ -24,13 +24,13 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -41,15 +41,12 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
-import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.MockResourceCalculatorPlugin;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.MockResourceCalculatorProcessTree;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.Assert;
 import org.junit.Test;
@@ -58,11 +55,6 @@ import org.junit.Test;
  * Class for testing the {@link QueuingContainerManagerImpl}.
  */
 public class TestQueuingContainerManager extends BaseContainerManagerTest {
-
-  interface HasResources {
-    boolean decide(Context context, ContainerId cId);
-  }
-
   public TestQueuingContainerManager() throws UnsupportedFileSystemException {
     super();
   }
@@ -78,18 +70,6 @@ public class TestQueuingContainerManager extends BaseContainerManagerTest {
       DeletionService delSrvc) {
     return new QueuingContainerManagerImpl(context, exec, delSrvc,
         nodeStatusUpdater, metrics, dirsHandler) {
-
-      @Override
-      public void serviceInit(Configuration conf) throws Exception {
-        conf.set(
-            YarnConfiguration.NM_CONTAINER_MON_RESOURCE_CALCULATOR,
-            MockResourceCalculatorPlugin.class.getCanonicalName());
-        conf.set(
-            YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE,
-            MockResourceCalculatorProcessTree.class.getCanonicalName());
-        super.serviceInit(conf);
-      }
-
       @Override
       public void
       setBlockNewContainerRequests(boolean blockNewContainerRequests) {
@@ -398,7 +378,7 @@ public class TestQueuingContainerManager extends BaseContainerManagerTest {
     containerManager.startContainers(allRequests);
 
     BaseContainerManagerTest.waitForNMContainerState(containerManager,
-        createContainerId(0), ContainerState.DONE, 30);
+        createContainerId(0), ContainerState.DONE, 40);
     Thread.sleep(5000);
 
     // Get container statuses. Container 0 should be killed, container 1
@@ -429,7 +409,7 @@ public class TestQueuingContainerManager extends BaseContainerManagerTest {
 
     // Make sure the remaining OPPORTUNISTIC container starts its execution.
     BaseContainerManagerTest.waitForNMContainerState(containerManager,
-        createContainerId(2), ContainerState.DONE, 30);
+        createContainerId(2), ContainerState.DONE, 40);
     Thread.sleep(5000);
     statRequest = GetContainerStatusesRequest.newInstance(Arrays.asList(
         createContainerId(1)));
@@ -488,13 +468,12 @@ public class TestQueuingContainerManager extends BaseContainerManagerTest {
     containerManager.startContainers(allRequests);
 
     BaseContainerManagerTest.waitForNMContainerState(containerManager,
-        createContainerId(0), ContainerState.DONE, 30);
+        createContainerId(0), ContainerState.DONE, 40);
     Thread.sleep(5000);
 
     // Get container statuses. Container 0 should be killed, container 1
     // should be queued and container 2 should be running.
     int killedContainers = 0;
-    int runningContainers = 0;
     List<ContainerId> statList = new ArrayList<ContainerId>();
     for (int i = 0; i < 4; i++) {
       statList.add(createContainerId(i));
@@ -508,14 +487,108 @@ public class TestQueuingContainerManager extends BaseContainerManagerTest {
           "Container killed by the ApplicationMaster")) {
         killedContainers++;
       }
+      System.out.println("\nStatus : [" + status + "]\n");
+    }
+
+    Assert.assertEquals(2, killedContainers);
+  }
+
+  /**
+   * Start running one GUARANTEED container and queue two OPPORTUNISTIC ones.
+   * Try killing one of the two queued containers.
+   * @throws Exception
+   */
+  @Test
+  public void testStopQueuedContainer() throws Exception {
+    shouldDeleteWait = true;
+    containerManager.start();
+
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(2048, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.GUARANTEED)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(512, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(2), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(512, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+
+    Thread.sleep(2000);
+
+    // Assert there is initially one container running and two queued.
+    int runningContainersNo = 0;
+    int queuedContainersNo = 0;
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    for (int i = 0; i < 3; i++) {
+      statList.add(createContainerId(i));
+    }
+    GetContainerStatusesRequest statRequest = GetContainerStatusesRequest
+        .newInstance(statList);
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    for (ContainerStatus status : containerStatuses) {
       if (status.getState() ==
           org.apache.hadoop.yarn.api.records.ContainerState.RUNNING) {
-        runningContainers++;
+        runningContainersNo++;
+      } else if (status.getState() ==
+          org.apache.hadoop.yarn.api.records.ContainerState.QUEUED) {
+        queuedContainersNo++;
       }
       System.out.println("\nStatus : [" + status + "]\n");
     }
 
-    Assert.assertEquals(2, killedContainers);
-    Assert.assertEquals(2, runningContainers);
+    Assert.assertEquals(1, runningContainersNo);
+    Assert.assertEquals(2, queuedContainersNo);
+
+    // Stop one of the two queued containers.
+    StopContainersRequest stopRequest = StopContainersRequest.
+        newInstance(Arrays.asList(createContainerId(1)));
+    containerManager.stopContainers(stopRequest);
+
+    Thread.sleep(2000);
+
+    // Assert queued container got properly stopped.
+    statList.clear();
+    for (int i = 0; i < 3; i++) {
+      statList.add(createContainerId(i));
+    }
+    statRequest = GetContainerStatusesRequest.newInstance(statList);
+    containerStatuses = containerManager.getContainerStatuses(statRequest)
+        .getContainerStatuses();
+    for (ContainerStatus status : containerStatuses) {
+      if (status.getContainerId().equals(createContainerId(0))) {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+            status.getState());
+      } else if (status.getContainerId().equals(createContainerId(1))) {
+        Assert.assertTrue(status.getDiagnostics().contains(
+            "Queued container request removed"));
+      } else if (status.getContainerId().equals(createContainerId(2))) {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.QUEUED,
+            status.getState());
+      }
+      System.out.println("\nStatus : [" + status + "]\n");
+    }
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[32/51] [abbrv] hadoop git commit: YARN-3426. Add jdiff support to YARN. (vinodkv via wangda)

Posted by vv...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/03fc6b1b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client.2.6.0.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client.2.6.0.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client.2.6.0.xml
new file mode 100644
index 0000000..aa11aea
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client.2.6.0.xml
@@ -0,0 +1,2427 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!--
+   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.
+-->
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Wed Apr 08 11:30:44 PDT 2015 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="hadoop-yarn-client 2.6.0"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.ExcludePrivateAnnotationsJDiffDoclet -docletpath /Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/hadoop-annotations.jar:/Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/jdiff.jar -verbose -classpath /Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/classes:/Users/llu/.m2/repository/org/apache/hadoop/hadoop-common/2.6.0/hadoop-common-2.6.0.jar:/Users/llu/.m2/repository/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/Users/llu/.m2/repository/xmlenc/xmlenc/0.52/xmlenc-0.52.jar:/Users/llu/.m2/repository/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1.jar:/Users/llu/.m2/repository/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/Users/llu/.m2/repository/commons-io/commons-io/2.4/commons-io-2.4.jar:/Users/llu/.m2/repository/commons-net/commons-net/3.1/commons-net-3.1.jar:/Users/llu/.m
 2/repository/commons-collections/commons-collections/3.2.1/commons-collections-3.2.1.jar:/Users/llu/.m2/repository/javax/servlet/servlet-api/2.5/servlet-api-2.5.jar:/Users/llu/.m2/repository/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/Users/llu/.m2/repository/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/Users/llu/.m2/repository/com/sun/jersey/jersey-core/1.9/jersey-core-1.9.jar:/Users/llu/.m2/repository/com/sun/jersey/jersey-json/1.9/jersey-json-1.9.jar:/Users/llu/.m2/repository/com/sun/xml/bind/jaxb-impl/2.2.3-1/jaxb-impl-2.2.3-1.jar:/Users/llu/.m2/repository/com/sun/jersey/jersey-server/1.9/jersey-server-1.9.jar:/Users/llu/.m2/repository/asm/asm/3.2/asm-3.2.jar:/Users/llu/.m2/repository/javax/servlet/jsp/jsp-api/2.1/jsp-api-2.1.jar:/Users/llu/.m2/repository/net/java/dev/jets3t/jets3t/0.9.0/jets3t-0.9.0.jar:/Users/llu/.m2/repository/org/apache/httpcomponents/httpclient/4.2.5/httpclient-4.2.5.jar:/Users/llu/.m2/repository/org/apache/httpcomponents/httpcore/4.2.5/h
 ttpcore-4.2.5.jar:/Users/llu/.m2/repository/com/jamesmurty/utils/java-xmlbuilder/0.4/java-xmlbuilder-0.4.jar:/Users/llu/.m2/repository/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/Users/llu/.m2/repository/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/Users/llu/.m2/repository/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/Users/llu/.m2/repository/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar:/Users/llu/.m2/repository/org/slf4j/slf4j-api/1.7.5/slf4j-api-1.7.5.jar:/Users/llu/.m2/repository/org/slf4j/slf4j-log4j12/1.7.5/slf4j-log4j12-1.7.5.jar:/Users/llu/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/llu/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/Users/llu/.m2/repository/org/apache/avro/avro/1.7.4/avro-1.7.4.jar:/Users/llu/.m2/repository/com/thoughtworks/paranamer/paranamer/2.3/para
 namer-2.3.jar:/Users/llu/.m2/repository/org/xerial/snappy/snappy-java/1.0.4.1/snappy-java-1.0.4.1.jar:/Users/llu/.m2/repository/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar:/Users/llu/.m2/repository/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/Users/llu/.m2/repository/org/apache/hadoop/hadoop-auth/2.6.0/hadoop-auth-2.6.0.jar:/Users/llu/.m2/repository/org/apache/directory/server/apacheds-kerberos-codec/2.0.0-M15/apacheds-kerberos-codec-2.0.0-M15.jar:/Users/llu/.m2/repository/org/apache/directory/server/apacheds-i18n/2.0.0-M15/apacheds-i18n-2.0.0-M15.jar:/Users/llu/.m2/repository/org/apache/directory/api/api-asn1-api/1.0.0-M20/api-asn1-api-1.0.0-M20.jar:/Users/llu/.m2/repository/org/apache/directory/api/api-util/1.0.0-M20/api-util-1.0.0-M20.jar:/Users/llu/.m2/repository/org/apache/curator/curator-framework/2.6.0/curator-framework-2.6.0.jar:/Users/llu/.m2/repository/com/jcraft/jsch/0.1.42/jsch-0.1.42.jar:/Users/llu/.m2/repository/org/apache/curator/curator-client/
 2.6.0/curator-client-2.6.0.jar:/Users/llu/.m2/repository/org/apache/curator/curator-recipes/2.6.0/curator-recipes-2.6.0.jar:/Users/llu/.m2/repository/com/google/code/findbugs/jsr305/1.3.9/jsr305-1.3.9.jar:/Users/llu/.m2/repository/org/htrace/htrace-core/3.0.4/htrace-core-3.0.4.jar:/Users/llu/.m2/repository/org/apache/zookeeper/zookeeper/3.4.6/zookeeper-3.4.6.jar:/Users/llu/.m2/repository/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/Users/llu/.m2/repository/org/tukaani/xz/1.0/xz-1.0.jar:/Users/llu/.m2/repository/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/Users/llu/.m2/repository/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/Users/llu/.m2/repository/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/Users/llu/.m2/repository/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/Users/llu/.m2/repository/log4j/log4j/1.2.17/log4j-1.2.17.jar:/Users/llu/.m2/repository/org/apache/hadoop/hadoop-annotations/2.6.0/hadoop-annotations-2.6.0.jar:/Librar
 y/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/lib/tools.jar:/Users/llu/.m2/repository/io/netty/netty/3.6.2.Final/netty-3.6.2.Final.jar:/Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/hadoop-yarn-api-2.6.0.jar:/Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/hadoop-yarn-common-2.6.0.jar:/Users/llu/.m2/repository/javax/xml/bind/jaxb-api/2.2.2/jaxb-api-2.2.2.jar:/Users/llu/.m2/repository/javax/xml/stream/stax-api/1.0-2/stax-api-1.0-2.jar:/Users/llu/.m2/repository/javax/activation/activation/1.1/activation-1.1.jar:/Users/llu/.m2/repository/com/sun/jersey/jersey-client/1.9/jersey-client-1.9.jar:/Users/llu/.m2/repository/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/Users/llu/.m2/repository/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/Users/llu/.m2/repository/com/google/inject/extensions/guice-servlet/3.0/guice-servlet-3.0.jar:/Users/llu/.m2/repository/com/google/inject/guice/3.0/gu
 ice-3.0.jar:/Users/llu/.m2/repository/javax/inject/javax.inject/1/javax.inject-1.jar:/Users/llu/.m2/repository/aopalliance/aopalliance/1.0/aopalliance-1.0.jar:/Users/llu/.m2/repository/com/sun/jersey/contribs/jersey-guice/1.9/jersey-guice-1.9.jar:/Users/llu/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar -sourcepath /Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java -apidir /Users/llu/hadoop2_6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/site/jdiff/xml -apiname hadoop-yarn-client 2.6.0 -->
+<package name="org.apache.hadoop.yarn.client.api">
+  <!-- start class org.apache.hadoop.yarn.client.api.AHSClient -->
+  <class name="AHSClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AHSClient" type="java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createAHSClient" return="org.apache.hadoop.yarn.client.api.AHSClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new instance of AHSClient.]]>
+      </doc>
+    </method>
+    <method name="getApplicationReport" return="org.apache.hadoop.yarn.api.records.ApplicationReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given Application.
+ </p>
+
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access then the following
+ fields in the report will be set to stubbed values:
+ <ul>
+ <li>host - set to "N/A"</li>
+ <li>RPC port - set to -1</li>
+ <li>client token - set to "N/A"</li>
+ <li>diagnostics - set to "N/A"</li>
+ <li>tracking URL - set to "N/A"</li>
+ <li>original tracking URL - set to "N/A"</li>
+ <li>resource usage report - all values are -1</li>
+ </ul>
+ </p>
+
+ @param appId
+          {@link ApplicationId} of the application that needs a report
+ @return application report
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of all Applications in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @return a list of reports for all applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttemptReport" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given ApplicationAttempt.
+ </p>
+
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+
+ @param applicationAttemptId
+          {@link ApplicationAttemptId} of the application attempt that needs
+          a report
+ @return application attempt report
+ @throws YarnException
+ @throws {@link ApplicationAttemptNotFoundException} if application attempt
+         not found
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttempts" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of all (ApplicationAttempts) of Application in the cluster.
+ </p>
+
+ @param applicationId
+ @return a list of reports for all application attempts for specified
+         application
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainerReport" return="org.apache.hadoop.yarn.api.records.ContainerReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given Container.
+ </p>
+
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+
+ @param containerId
+          {@link ContainerId} of the container that needs a report
+ @return container report
+ @throws YarnException
+ @throws {@link ContainerNotFoundException} if container not found
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainers" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of all (Containers) of ApplicationAttempt in the cluster.
+ </p>
+
+ @param applicationAttemptId
+ @return a list of reports of all containers for specified application
+         attempt
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.AHSClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.AMRMClient -->
+  <class name="AMRMClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AMRMClient" type="java.lang.String"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createAMRMClient" return="org.apache.hadoop.yarn.client.api.AMRMClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new instance of AMRMClient.
+ For usage:
+ <pre>
+ {@code
+ AMRMClient.<T>createAMRMClientContainerRequest()
+ }</pre>
+ @return the newly create AMRMClient instance.]]>
+      </doc>
+    </method>
+    <method name="registerApplicationMaster" return="org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appHostName" type="java.lang.String"/>
+      <param name="appHostPort" type="int"/>
+      <param name="appTrackingUrl" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Register the application master. This must be called before any
+ other interaction
+ @param appHostName Name of the host on which master is running
+ @param appHostPort Port master is listening on
+ @param appTrackingUrl URL at which the master info can be seen
+ @return <code>RegisterApplicationMasterResponse</code>
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="allocate" return="org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="progressIndicator" type="float"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Request additional containers and receive new container allocations.
+ Requests made via <code>addContainerRequest</code> are sent to the
+ <code>ResourceManager</code>. New containers assigned to the master are
+ retrieved. Status of completed containers and node health updates are also
+ retrieved. This also doubles up as a heartbeat to the ResourceManager and
+ must be made periodically. The call may not always return any new
+ allocations of containers. App should not make concurrent allocate
+ requests. May cause request loss.
+
+ <p>
+ Note : If the user has not removed container requests that have already
+ been satisfied, then the re-register may end up sending the entire
+ container requests to the RM (including matched requests). Which would mean
+ the RM could end up giving it a lot of new allocated containers.
+ </p>
+
+ @param progressIndicator Indicates progress made by the master
+ @return the response of the allocate request
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="unregisterApplicationMaster"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appStatus" type="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"/>
+      <param name="appMessage" type="java.lang.String"/>
+      <param name="appTrackingUrl" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Unregister the application master. This must be called in the end.
+ @param appStatus Success/Failure status of the master
+ @param appMessage Diagnostics message on failure
+ @param appTrackingUrl New URL to get master info
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="addContainerRequest"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="T"/>
+      <doc>
+      <![CDATA[Request containers for resources before calling <code>allocate</code>
+ @param req Resource request]]>
+      </doc>
+    </method>
+    <method name="removeContainerRequest"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="T"/>
+      <doc>
+      <![CDATA[Remove previous container request. The previous container request may have
+ already been sent to the ResourceManager. So even after the remove request
+ the app must be prepared to receive an allocation for the previous request
+ even after the remove request
+ @param req Resource request]]>
+      </doc>
+    </method>
+    <method name="releaseAssignedContainer"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <doc>
+      <![CDATA[Release containers assigned by the Resource Manager. If the app cannot use
+ the container or wants to give up the container then it can release them.
+ The app needs to make new requests for the released resource capability if
+ it still needs it. eg. it released non-local resources
+ @param containerId]]>
+      </doc>
+    </method>
+    <method name="getAvailableResources" return="org.apache.hadoop.yarn.api.records.Resource"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the currently available resources in the cluster.
+ A valid value is available after a call to allocate has been made
+ @return Currently available resources]]>
+      </doc>
+    </method>
+    <method name="getClusterNodeCount" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the current number of nodes in the cluster.
+ A valid values is available after a call to allocate has been made
+ @return Current number of nodes in the cluster]]>
+      </doc>
+    </method>
+    <method name="getMatchingRequests" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="priority" type="org.apache.hadoop.yarn.api.records.Priority"/>
+      <param name="resourceName" type="java.lang.String"/>
+      <param name="capability" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <doc>
+      <![CDATA[Get outstanding <code>ContainerRequest</code>s matching the given
+ parameters. These ContainerRequests should have been added via
+ <code>addContainerRequest</code> earlier in the lifecycle. For performance,
+ the AMRMClient may return its internal collection directly without creating
+ a copy. Users should not perform mutable operations on the return value.
+ Each collection in the list contains requests with identical
+ <code>Resource</code> size that fit in the given capability. In a
+ collection, requests will be returned in the same order as they were added.
+ @return Collection of request matching the parameters]]>
+      </doc>
+    </method>
+    <method name="updateBlacklist"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="blacklistAdditions" type="java.util.List"/>
+      <param name="blacklistRemovals" type="java.util.List"/>
+      <doc>
+      <![CDATA[Update application's blacklist with addition or removal resources.
+
+ @param blacklistAdditions list of resources which should be added to the
+        application blacklist
+ @param blacklistRemovals list of resources which should be removed from the
+        application blacklist]]>
+      </doc>
+    </method>
+    <method name="setNMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nmTokenCache" type="org.apache.hadoop.yarn.client.api.NMTokenCache"/>
+      <doc>
+      <![CDATA[Set the NM token cache for the <code>AMRMClient</code>. This cache must
+ be shared with the {@link NMClient} used to manage containers for the
+ <code>AMRMClient</code>
+ <p/>
+ If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
+ singleton instance will be used.
+
+ @param nmTokenCache the NM token cache to use.]]>
+      </doc>
+    </method>
+    <method name="getNMTokenCache" return="org.apache.hadoop.yarn.client.api.NMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the NM token cache of the <code>AMRMClient</code>. This cache must be
+ shared with the {@link NMClient} used to manage containers for the
+ <code>AMRMClient</code>.
+ <p/>
+ If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
+ singleton instance will be used.
+
+ @return the NM token cache.]]>
+      </doc>
+    </method>
+    <method name="waitFor"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="check" type="com.google.common.base.Supplier"/>
+      <exception name="InterruptedException" type="java.lang.InterruptedException"/>
+      <doc>
+      <![CDATA[Wait for <code>check</code> to return true for each 1000 ms.
+ See also {@link #waitFor(com.google.common.base.Supplier, int)}
+ and {@link #waitFor(com.google.common.base.Supplier, int, int)}
+ @param check]]>
+      </doc>
+    </method>
+    <method name="waitFor"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="check" type="com.google.common.base.Supplier"/>
+      <param name="checkEveryMillis" type="int"/>
+      <exception name="InterruptedException" type="java.lang.InterruptedException"/>
+      <doc>
+      <![CDATA[Wait for <code>check</code> to return true for each
+ <code>checkEveryMillis</code> ms.
+ See also {@link #waitFor(com.google.common.base.Supplier, int, int)}
+ @param check user defined checker
+ @param checkEveryMillis interval to call <code>check</code>]]>
+      </doc>
+    </method>
+    <method name="waitFor"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="check" type="com.google.common.base.Supplier"/>
+      <param name="checkEveryMillis" type="int"/>
+      <param name="logInterval" type="int"/>
+      <exception name="InterruptedException" type="java.lang.InterruptedException"/>
+      <doc>
+      <![CDATA[Wait for <code>check</code> to return true for each
+ <code>checkEveryMillis</code> ms. In the main loop, this method will log
+ the message "waiting in main loop" for each <code>logInterval</code> times
+ iteration to confirm the thread is alive.
+ @param check user defined checker
+ @param checkEveryMillis interval to call <code>check</code>
+ @param logInterval interval to log for each]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.AMRMClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -->
+  <class name="AMRMClient.ContainerRequest" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AMRMClient.ContainerRequest" type="org.apache.hadoop.yarn.api.records.Resource, java.lang.String[], java.lang.String[], org.apache.hadoop.yarn.api.records.Priority"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Instantiates a {@link ContainerRequest} with the given constraints and
+ locality relaxation enabled.
+
+ @param capability
+          The {@link Resource} to be requested for each container.
+ @param nodes
+          Any hosts to request that the containers are placed on.
+ @param racks
+          Any racks to request that the containers are placed on. The
+          racks corresponding to any hosts requested will be automatically
+          added to this list.
+ @param priority
+          The priority at which to request the containers. Higher
+          priorities have lower numerical values.]]>
+      </doc>
+    </constructor>
+    <constructor name="AMRMClient.ContainerRequest" type="org.apache.hadoop.yarn.api.records.Resource, java.lang.String[], java.lang.String[], org.apache.hadoop.yarn.api.records.Priority, boolean"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Instantiates a {@link ContainerRequest} with the given constraints.
+
+ @param capability
+          The {@link Resource} to be requested for each container.
+ @param nodes
+          Any hosts to request that the containers are placed on.
+ @param racks
+          Any racks to request that the containers are placed on. The
+          racks corresponding to any hosts requested will be automatically
+          added to this list.
+ @param priority
+          The priority at which to request the containers. Higher
+          priorities have lower numerical values.
+ @param relaxLocality
+          If true, containers for this request may be assigned on hosts
+          and racks other than the ones explicitly requested.]]>
+      </doc>
+    </constructor>
+    <constructor name="AMRMClient.ContainerRequest" type="org.apache.hadoop.yarn.api.records.Resource, java.lang.String[], java.lang.String[], org.apache.hadoop.yarn.api.records.Priority, boolean, java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Instantiates a {@link ContainerRequest} with the given constraints.
+
+ @param capability
+          The {@link Resource} to be requested for each container.
+ @param nodes
+          Any hosts to request that the containers are placed on.
+ @param racks
+          Any racks to request that the containers are placed on. The
+          racks corresponding to any hosts requested will be automatically
+          added to this list.
+ @param priority
+          The priority at which to request the containers. Higher
+          priorities have lower numerical values.
+ @param relaxLocality
+          If true, containers for this request may be assigned on hosts
+          and racks other than the ones explicitly requested.
+ @param nodeLabelsExpression
+          Set node labels to allocate resource]]>
+      </doc>
+    </constructor>
+    <method name="getCapability" return="org.apache.hadoop.yarn.api.records.Resource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getNodes" return="java.util.List"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getRacks" return="java.util.List"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getPriority" return="org.apache.hadoop.yarn.api.records.Priority"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getRelaxLocality" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getNodeLabelExpression" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <doc>
+    <![CDATA[Object to represent a single container request for resources. Scheduler
+ documentation should be consulted for the specifics of how the parameters
+ are honored.
+
+ By default, YARN schedulers try to allocate containers at the requested
+ locations but they may relax the constraints in order to expedite meeting
+ allocations limits. They first relax the constraint to the same rack as the
+ requested node and then to anywhere in the cluster. The relaxLocality flag
+ may be used to disable locality relaxation and request containers at only
+ specific locations. The following conditions apply.
+ <ul>
+ <li>Within a priority, all container requests must have the same value for
+ locality relaxation. Either enabled or disabled.</li>
+ <li>If locality relaxation is disabled, then across requests, locations at
+ different network levels may not be specified. E.g. its invalid to make a
+ request for a specific node and another request for a specific rack.</li>
+ <li>If locality relaxation is disabled, then only within the same request,
+ a node and its rack may be specified together. This allows for a specific
+ rack with a preference for a specific node within that rack.</li>
+ <li></li>
+ </ul>
+ To re-enable locality relaxation at a given priority, all pending requests
+ with locality relaxation disabled must be first removed. Then they can be
+ added back with locality relaxation enabled.
+
+ All getters return immutable values.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -->
+  <!-- start class org.apache.hadoop.yarn.client.api.InvalidContainerRequestException -->
+  <class name="InvalidContainerRequestException" extends="org.apache.hadoop.yarn.exceptions.YarnRuntimeException"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="InvalidContainerRequestException" type="java.lang.Throwable"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="InvalidContainerRequestException" type="java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="InvalidContainerRequestException" type="java.lang.String, java.lang.Throwable"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <doc>
+    <![CDATA[Thrown when an arguments are combined to construct a
+ <code>AMRMClient.ContainerRequest</code> in an invalid way.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.InvalidContainerRequestException -->
+  <!-- start class org.apache.hadoop.yarn.client.api.NMClient -->
+  <class name="NMClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NMClient" type="java.lang.String"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createNMClient" return="org.apache.hadoop.yarn.client.api.NMClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new instance of NMClient.]]>
+      </doc>
+    </method>
+    <method name="createNMClient" return="org.apache.hadoop.yarn.client.api.NMClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Create a new instance of NMClient.]]>
+      </doc>
+    </method>
+    <method name="startContainer" return="java.util.Map"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="container" type="org.apache.hadoop.yarn.api.records.Container"/>
+      <param name="containerLaunchContext" type="org.apache.hadoop.yarn.api.records.ContainerLaunchContext"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Start an allocated container.</p>
+
+ <p>The <code>ApplicationMaster</code> or other applications that use the
+ client must provide the details of the allocated container, including the
+ Id, the assigned node's Id and the token via {@link Container}. In
+ addition, the AM needs to provide the {@link ContainerLaunchContext} as
+ well.</p>
+
+ @param container the allocated container
+ @param containerLaunchContext the context information needed by the
+                               <code>NodeManager</code> to launch the
+                               container
+ @return a map between the auxiliary service names and their outputs
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="stopContainer"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Stop an started container.</p>
+
+ @param containerId the Id of the started container
+ @param nodeId the Id of the <code>NodeManager</code>
+
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainerStatus" return="org.apache.hadoop.yarn.api.records.ContainerStatus"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Query the status of a container.</p>
+
+ @param containerId the Id of the started container
+ @param nodeId the Id of the <code>NodeManager</code>
+
+ @return the status of a container
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="cleanupRunningContainersOnStop"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="enabled" type="boolean"/>
+      <doc>
+      <![CDATA[<p>Set whether the containers that are started by this client, and are
+ still running should be stopped when the client stops. By default, the
+ feature should be enabled.</p> However, containers will be stopped only
+ when service is stopped. i.e. after {@link NMClient#stop()}.
+
+ @param enabled whether the feature is enabled or not]]>
+      </doc>
+    </method>
+    <method name="setNMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nmTokenCache" type="org.apache.hadoop.yarn.client.api.NMTokenCache"/>
+      <doc>
+      <![CDATA[Set the NM Token cache of the <code>NMClient</code>. This cache must be
+ shared with the {@link AMRMClient} that requested the containers managed
+ by this <code>NMClient</code>
+ <p/>
+ If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
+ singleton instance will be used.
+
+ @param nmTokenCache the NM token cache to use.]]>
+      </doc>
+    </method>
+    <method name="getNMTokenCache" return="org.apache.hadoop.yarn.client.api.NMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the NM token cache of the <code>NMClient</code>. This cache must be
+ shared with the {@link AMRMClient} that requested the containers managed
+ by this <code>NMClient</code>
+ <p/>
+ If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
+ singleton instance will be used.
+
+ @return the NM token cache]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.NMClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.NMTokenCache -->
+  <class name="NMTokenCache" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NMTokenCache"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Creates a NM token cache instance.]]>
+      </doc>
+    </constructor>
+    <method name="getSingleton" return="org.apache.hadoop.yarn.client.api.NMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Returns the singleton NM token cache.
+
+ @return the singleton NM token cache.]]>
+      </doc>
+    </method>
+    <method name="getNMToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeAddr" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Returns NMToken, null if absent. Only the singleton obtained from
+ {@link #getSingleton()} is looked at for the tokens. If you are using your
+ own NMTokenCache that is different from the singleton, use
+ {@link #getToken(String) }
+
+ @param nodeAddr
+ @return {@link Token} NMToken required for communicating with node manager]]>
+      </doc>
+    </method>
+    <method name="setNMToken"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeAddr" type="java.lang.String"/>
+      <param name="token" type="org.apache.hadoop.yarn.api.records.Token"/>
+      <doc>
+      <![CDATA[Sets the NMToken for node address only in the singleton obtained from
+ {@link #getSingleton()}. If you are using your own NMTokenCache that is
+ different from the singleton, use {@link #setToken(String, Token) }
+
+ @param nodeAddr
+          node address (host:port)
+ @param token
+          NMToken]]>
+      </doc>
+    </method>
+    <method name="getToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeAddr" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Returns NMToken, null if absent
+ @param nodeAddr
+ @return {@link Token} NMToken required for communicating with node
+         manager]]>
+      </doc>
+    </method>
+    <method name="setToken"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeAddr" type="java.lang.String"/>
+      <param name="token" type="org.apache.hadoop.yarn.api.records.Token"/>
+      <doc>
+      <![CDATA[Sets the NMToken for node address
+ @param nodeAddr node address (host:port)
+ @param token NMToken]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[NMTokenCache manages NMTokens required for an Application Master
+ communicating with individual NodeManagers.
+ <p/>
+ By default Yarn client libraries {@link AMRMClient} and {@link NMClient} use
+ {@link #getSingleton()} instance of the cache.
+ <ul>
+ <li>Using the singleton instance of the cache is appropriate when running a
+ single ApplicationMaster in the same JVM.</li>
+ <li>When using the singleton, users don't need to do anything special,
+ {@link AMRMClient} and {@link NMClient} are already set up to use the default
+ singleton {@link NMTokenCache}</li>
+ </ul>
+ <p/>
+ If running multiple Application Masters in the same JVM, a different cache
+ instance should be used for each Application Master.
+ <p/>
+ <ul>
+ <li>
+ If using the {@link AMRMClient} and the {@link NMClient}, setting up and using
+ an instance cache is as follows:
+ <p/>
+
+ <pre>
+   NMTokenCache nmTokenCache = new NMTokenCache();
+   AMRMClient rmClient = AMRMClient.createAMRMClient();
+   NMClient nmClient = NMClient.createNMClient();
+   nmClient.setNMTokenCache(nmTokenCache);
+   ...
+ </pre>
+ </li>
+ <li>
+ If using the {@link AMRMClientAsync} and the {@link NMClientAsync}, setting up
+ and using an instance cache is as follows:
+ <p/>
+
+ <pre>
+   NMTokenCache nmTokenCache = new NMTokenCache();
+   AMRMClient rmClient = AMRMClient.createAMRMClient();
+   NMClient nmClient = NMClient.createNMClient();
+   nmClient.setNMTokenCache(nmTokenCache);
+   AMRMClientAsync rmClientAsync = new AMRMClientAsync(rmClient, 1000, [AMRM_CALLBACK]);
+   NMClientAsync nmClientAsync = new NMClientAsync("nmClient", nmClient, [NM_CALLBACK]);
+   ...
+ </pre>
+ </li>
+ <li>
+ If using {@link ApplicationMasterProtocol} and
+ {@link ContainerManagementProtocol} directly, setting up and using an
+ instance cache is as follows:
+ <p/>
+
+ <pre>
+   NMTokenCache nmTokenCache = new NMTokenCache();
+   ...
+   ApplicationMasterProtocol amPro = ClientRMProxy.createRMProxy(conf, ApplicationMasterProtocol.class);
+   ...
+   AllocateRequest allocateRequest = ...
+   ...
+   AllocateResponse allocateResponse = rmClient.allocate(allocateRequest);
+   for (NMToken token : allocateResponse.getNMTokens()) {
+     nmTokenCache.setToken(token.getNodeId().toString(), token.getToken());
+   }
+   ...
+   ContainerManagementProtocolProxy nmPro = ContainerManagementProtocolProxy(conf, nmTokenCache);
+   ...
+   nmPro.startContainer(container, containerContext);
+   ...
+ </pre>
+ </li>
+ </ul>
+ It is also possible to mix the usage of a client (<code>AMRMClient</code> or
+ <code>NMClient</code>, or the async versions of them) with a protocol proxy (
+ <code>ContainerManagementProtocolProxy</code> or
+ <code>ApplicationMasterProtocol</code>).]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.NMTokenCache -->
+  <!-- start class org.apache.hadoop.yarn.client.api.YarnClient -->
+  <class name="YarnClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="YarnClient" type="java.lang.String"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createYarnClient" return="org.apache.hadoop.yarn.client.api.YarnClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new instance of YarnClient.]]>
+      </doc>
+    </method>
+    <method name="createApplication" return="org.apache.hadoop.yarn.client.api.YarnClientApplication"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Obtain a {@link YarnClientApplication} for a new application,
+ which in turn contains the {@link ApplicationSubmissionContext} and
+ {@link org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse}
+ objects.
+ </p>
+
+ @return {@link YarnClientApplication} built for a new application
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="submitApplication" return="org.apache.hadoop.yarn.api.records.ApplicationId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appContext" type="org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Submit a new application to <code>YARN.</code> It is a blocking call - it
+ will not return {@link ApplicationId} until the submitted application is
+ submitted successfully and accepted by the ResourceManager.
+ </p>
+
+ <p>
+ Users should provide an {@link ApplicationId} as part of the parameter
+ {@link ApplicationSubmissionContext} when submitting a new application,
+ otherwise it will throw the {@link ApplicationIdNotProvidedException}.
+ </p>
+
+ <p>This internally calls {@link ApplicationClientProtocol#submitApplication
+ (SubmitApplicationRequest)}, and after that, it internally invokes
+ {@link ApplicationClientProtocol#getApplicationReport
+ (GetApplicationReportRequest)} and waits till it can make sure that the
+ application gets properly submitted. If RM fails over or RM restart
+ happens before ResourceManager saves the application's state,
+ {@link ApplicationClientProtocol
+ #getApplicationReport(GetApplicationReportRequest)} will throw
+ the {@link ApplicationNotFoundException}. This API automatically resubmits
+ the application with the same {@link ApplicationSubmissionContext} when it
+ catches the {@link ApplicationNotFoundException}</p>
+
+ @param appContext
+          {@link ApplicationSubmissionContext} containing all the details
+          needed to submit a new application
+ @return {@link ApplicationId} of the accepted application
+ @throws YarnException
+ @throws IOException
+ @see #createApplication()]]>
+      </doc>
+    </method>
+    <method name="killApplication"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Kill an application identified by given ID.
+ </p>
+
+ @param applicationId
+          {@link ApplicationId} of the application that needs to be killed
+ @throws YarnException
+           in case of errors or if YARN rejects the request due to
+           access-control restrictions.
+ @throws IOException
+ @see #getQueueAclsInfo()]]>
+      </doc>
+    </method>
+    <method name="getApplicationReport" return="org.apache.hadoop.yarn.api.records.ApplicationReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given Application.
+ </p>
+
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access then the following
+ fields in the report will be set to stubbed values:
+ <ul>
+ <li>host - set to "N/A"</li>
+ <li>RPC port - set to -1</li>
+ <li>client token - set to "N/A"</li>
+ <li>diagnostics - set to "N/A"</li>
+ <li>tracking URL - set to "N/A"</li>
+ <li>original tracking URL - set to "N/A"</li>
+ <li>resource usage report - all values are -1</li>
+ </ul>
+ </p>
+
+ @param appId
+          {@link ApplicationId} of the application that needs a report
+ @return application report
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getAMRMToken" return="org.apache.hadoop.security.token.Token"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Get the AMRM token of the application.
+ <p/>
+ The AMRM token is required for AM to RM scheduling operations. For
+ managed Application Masters Yarn takes care of injecting it. For unmanaged
+ Applications Masters, the token must be obtained via this method and set
+ in the {@link org.apache.hadoop.security.UserGroupInformation} of the
+ current user.
+ <p/>
+ The AMRM token will be returned only if all the following conditions are
+ met:
+ <li>
+   <ul>the requester is the owner of the ApplicationMaster</ul>
+   <ul>the application master is an unmanaged ApplicationMaster</ul>
+   <ul>the application master is in ACCEPTED state</ul>
+ </li>
+ Else this method returns NULL.
+
+ @param appId {@link ApplicationId} of the application to get the AMRM token
+ @return the AMRM token if available
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of all Applications in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @return a list of reports of all running applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationTypes" type="java.util.Set"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of Applications
+ matching the given application types in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @param applicationTypes
+ @return a list of reports of applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationStates" type="java.util.EnumSet"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of Applications matching the given
+ application states in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @param applicationStates
+ @return a list of reports of applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationTypes" type="java.util.Set"/>
+      <param name="applicationStates" type="java.util.EnumSet"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of Applications matching the given
+ application types and application states in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @param applicationTypes
+ @param applicationStates
+ @return a list of reports of applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getYarnClusterMetrics" return="org.apache.hadoop.yarn.api.records.YarnClusterMetrics"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get metrics ({@link YarnClusterMetrics}) about the cluster.
+ </p>
+
+ @return cluster metrics
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getNodeReports" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="states" type="org.apache.hadoop.yarn.api.records.NodeState[]"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of nodes ({@link NodeReport}) in the cluster.
+ </p>
+
+ @param states The {@link NodeState}s to filter on. If no filter states are
+          given, nodes in all states will be returned.
+ @return A list of node reports
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getRMDelegationToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="renewer" type="org.apache.hadoop.io.Text"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a delegation token so as to be able to talk to YARN using those tokens.
+
+ @param renewer
+          Address of the renewer who can renew these tokens when needed by
+          securely talking to YARN.
+ @return a delegation token ({@link Token}) that can be used to
+         talk to YARN
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getQueueInfo" return="org.apache.hadoop.yarn.api.records.QueueInfo"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="queueName" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information ({@link QueueInfo}) about a given <em>queue</em>.
+ </p>
+
+ @param queueName
+          Name of the queue whose information is needed
+ @return queue information
+ @throws YarnException
+           in case of errors or if YARN rejects the request due to
+           access-control restrictions.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getAllQueues" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information ({@link QueueInfo}) about all queues, recursively if there
+ is a hierarchy
+ </p>
+
+ @return a list of queue-information for all queues
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getRootQueueInfos" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information ({@link QueueInfo}) about top level queues.
+ </p>
+
+ @return a list of queue-information for all the top-level queues
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getChildQueueInfos" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="parent" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information ({@link QueueInfo}) about all the immediate children queues
+ of the given queue
+ </p>
+
+ @param parent
+          Name of the queue whose child-queues' information is needed
+ @return a list of queue-information for all queues who are direct children
+         of the given parent queue.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getQueueAclsInfo" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information about <em>acls</em> for <em>current user</em> on all the
+ existing queues.
+ </p>
+
+ @return a list of queue acls ({@link QueueUserACLInfo}) for
+         <em>current user</em>
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttemptReport" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given ApplicationAttempt.
+ </p>
+
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+
+ @param applicationAttemptId
+          {@link ApplicationAttemptId} of the application attempt that needs
+          a report
+ @return application attempt report
+ @throws YarnException
+ @throws {@link ApplicationAttemptNotFoundException} if application attempt
+         not found
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttempts" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of all (ApplicationAttempts) of Application in the cluster.
+ </p>
+
+ @param applicationId
+ @return a list of reports for all application attempts for specified
+         application.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainerReport" return="org.apache.hadoop.yarn.api.records.ContainerReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given Container.
+ </p>
+
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+
+ @param containerId
+          {@link ContainerId} of the container that needs a report
+ @return container report
+ @throws YarnException
+ @throws {@link ContainerNotFoundException} if container not found.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainers" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of all (Containers) of ApplicationAttempt in the cluster.
+ </p>
+
+ @param applicationAttemptId
+ @return a list of reports of all containers for specified application
+         attempts
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="moveApplicationAcrossQueues"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="queue" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Attempts to move the given application to the given queue.
+ </p>
+
+ @param appId
+    Application to move.
+ @param queue
+    Queue to place it in to.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="submitReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to submit a new reservation to the
+ {@code ResourceManager}.
+ </p>
+
+ <p>
+ The client packages all details of its request in a
+ {@link ReservationSubmissionRequest} object. This contains information
+ about the amount of capacity, temporal constraints, and gang needs.
+ Furthermore, the reservation might be composed of multiple stages, with
+ ordering dependencies among them.
+ </p>
+
+ <p>
+ In order to respond, a new admission control component in the
+ {@code ResourceManager} performs an analysis of the resources that have
+ been committed over the period of time the user is requesting, verify that
+ the user requests can be fulfilled, and that it respect a sharing policy
+ (e.g., {@code CapacityOverTimePolicy}). Once it has positively determined
+ that the ReservationRequest is satisfiable the {@code ResourceManager}
+ answers with a {@link ReservationSubmissionResponse} that includes a
+ {@link ReservationId}. Upon failure to find a valid allocation the response
+ is an exception with the message detailing the reason of failure.
+ </p>
+
+ <p>
+ The semantics guarantees that the {@link ReservationId} returned,
+ corresponds to a valid reservation existing in the time-range request by
+ the user. The amount of capacity dedicated to such reservation can vary
+ overtime, depending of the allocation that has been determined. But it is
+ guaranteed to satisfy all the constraint expressed by the user in the
+ {@link ReservationDefinition}
+ </p>
+
+ @param request request to submit a new Reservation
+ @return response contains the {@link ReservationId} on accepting the
+         submission
+ @throws YarnException if the reservation cannot be created successfully
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="updateReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to update an existing Reservation. This is
+ referred to as a re-negotiation process, in which a user that has
+ previously submitted a Reservation.
+ </p>
+
+ <p>
+ The allocation is attempted by virtually substituting all previous
+ allocations related to this Reservation with new ones, that satisfy the new
+ {@link ReservationDefinition}. Upon success the previous allocation is
+ atomically substituted by the new one, and on failure (i.e., if the system
+ cannot find a valid allocation for the updated request), the previous
+ allocation remains valid.
+ </p>
+
+ @param request to update an existing Reservation (the
+          {@link ReservationUpdateRequest} should refer to an existing valid
+          {@link ReservationId})
+ @return response empty on successfully updating the existing reservation
+ @throws YarnException if the request is invalid or reservation cannot be
+           updated successfully
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="deleteReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to remove an existing Reservation.
+ </p>
+
+ @param request to remove an existing Reservation (the
+          {@link ReservationDeleteRequest} should refer to an existing valid
+          {@link ReservationId})
+ @return response empty on successfully deleting the existing reservation
+ @throws YarnException if the request is invalid or reservation cannot be
+           deleted successfully
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getNodeToLabels" return="java.util.Map"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by client to get node to labels mappings in existing cluster
+ </p>
+
+ @return node to labels mappings
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getClusterNodeLabels" return="java.util.Set"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by client to get node labels in the cluster
+ </p>
+
+ @return cluster node labels collection
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.YarnClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.YarnClientApplication -->
+  <class name="YarnClientApplication" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="YarnClientApplication" type="org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse, org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getNewApplicationResponse" return="org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getApplicationSubmissionContext" return="org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.YarnClientApplication -->
+</package>
+<package name="org.apache.hadoop.yarn.client.api.async">
+  <!-- start class org.apache.hadoop.yarn.client.api.async.AMRMClientAsync -->
+  <class name="AMRMClientAsync" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AMRMClientAsync" type="int, org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AMRMClientAsync" type="org.apache.hadoop.yarn.client.api.AMRMClient, int, org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createAMRMClientAsync" return="org.apache.hadoop.yarn.client.api.async.AMRMClientAsync"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="intervalMs" type="int"/>
+      <param name="callbackHandler" type="org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler"/>
+    </method>
+    <method name="createAMRMClientAsync" return="org.apache.hadoop.yarn.client.api.async.AMRMClientAsync"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="client" type="org.apache.hadoop.yarn.client.api.AMRMClient"/>
+      <param name="intervalMs" type="int"/>
+      <param name="callbackHandler" type="org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler"/>
+    </method>
+    <method name="setHeartbeatInterval"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="interval" type="int"/>
+    </method>
+    <method name="getMatchingRequests" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="priority" type="org.apache.hadoop.yarn.api.records.Priority"/>
+      <param name="resourceName" type="java.lang.String"/>
+      <param name="capability" type="org.apache.hadoop.yarn.api.records.Resource"/>
+    </method>
+    <method name="registerApplicationMaster" return="org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appHostName" type="java.lang.String"/>
+      <param name="appHostPort" type="int"/>
+      <param name="appTrackingUrl" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Registers this application master with the resource manager. On successful
+ registration, starts the heartbeating thread.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="unregisterApplicationMaster"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appStatus" type="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"/>
+      <param name="appMessage" type="java.lang.String"/>
+      <param name="appTrackingUrl" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Unregister the application master. This must be called in the end.
+ @param appStatus Success/Failure status of the master
+ @param appMessage Diagnostics message on failure
+ @param appTrackingUrl New URL to get master info
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="addContainerRequest"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="T"/>
+      <doc>
+      <![CDATA[Request containers for resources before calling <code>allocate</code>
+ @param req Resource request]]>
+      </doc>
+    </method>
+    <method name="removeContainerRequest"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="T"/>
+      <doc>
+      <![CDATA[Remove previous container request. The previous container request may have
+ already been sent to the ResourceManager. So even after the remove request
+ the app must be prepared to receive an allocation for the previous request
+ even after the remove request
+ @param req Resource request]]>
+      </doc>
+    </method>
+    <method name="releaseAssignedContainer"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <doc>
+      <![CDATA[Release containers assigned by the Resource Manager. If the app cannot use
+ the container or wants to give up the container then it can release them.
+ The app needs to make new requests for the released resource capability if
+ it still needs it. eg. it released non-local resources
+ @param containerId]]>
+      </doc>
+    </method>
+    <method name="getAvailableResources" return="org.apache.hadoop.yarn.api.records.Resource"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the currently available resources in the cluster.
+ A valid value is available after a call to allocate has been made
+ @return Currently available resources]]>
+      </doc>
+    </method>
+    <method name="getClusterNodeCount" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the current number of nodes in the cluster.
+ A valid values is available after a call to allocate has been made
+ @return Current number of nodes in the cluster]]>
+      </doc>
+    </method>
+    <method name="waitFor"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="check" type="com.google.common.base.Supplier"/>
+      <exception name="InterruptedException" type="java.lang.InterruptedException"/>
+      <doc>
+      <![CDATA[Wait for <code>check</code> to return true for each 1000 ms.
+ See also {@link #waitFor(com.google.common.base.Supplier, int)}
+ and {@link #waitFor(com.google.common.base.Supplier, int, int)}
+ @param check]]>
+      </doc>
+    </method>
+    <method name="waitFor"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="check" type="com.google.common.base.Supplier"/>
+      <param name="checkEveryMillis" type="int"/>
+      <exception name="InterruptedException" type="java.lang.InterruptedException"/>
+      <doc>
+      <![CDATA[Wait for <code>check</code> to return true for each
+ <code>checkEveryMillis</code> ms.
+ See also {@link #waitFor(com.google.common.base.Supplier, int, int)}
+ @param check user defined checker
+ @param checkEveryMillis interval to call <code>check</code>]]>
+      </doc>
+    </method>
+    <method name="waitFor"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="check" type="com.google.common.base.Supplier"/>
+      <param name="checkEveryMillis" type="int"/>
+      <param name="logInterval" type="int"/>
+      <exception name="InterruptedException" type="java.lang.InterruptedException"/>
+      <doc>
+      <![CDATA[Wait for <code>check</code> to return true for each
+ <code>checkEveryMillis</code> ms. In the main loop, this method will log
+ the message "waiting in main loop" for each <code>logInterval</code> times
+ iteration to confirm the thread is alive.
+ @param check user defined checker
+ @param checkEveryMillis interval to call <code>check</code>
+ @param logInterval interval to log for each]]>
+      </doc>
+    </method>
+    <field name="client" type="org.apache.hadoop.yarn.client.api.AMRMClient"
+      transient="false" volatile="false"
+      static="false" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="handler" type="org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler"
+      transient="false" volatile="false"
+      static="false" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="heartbeatIntervalMs" type="java.util.concurrent.atomic.AtomicInteger"
+      transient="false" volatile="false"
+      static="false" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[<code>AMRMClientAsync</code> handles communication with the ResourceManager
+ and provides asynchronous updates on events such as container allocations and
+ completions.  It contains a thread that sends periodic heartbeats to the
+ ResourceManager.
+
+ It should be used by implementing a CallbackHandler:
+ <pre>
+ {@code
+ class MyCallbackHandler implements AMRMClientAsync.CallbackHandler {
+   public void onContainersAllocated(List<Container> containers) {
+     [run tasks on the containers]
+   }
+
+   public void onContainersCompleted(List<ContainerStatus> statuses) {
+     [update progress, check whether app is done]
+   }
+
+   public void onNodesUpdated(List<NodeReport> updated) {}
+
+   public void onReboot() {}
+ }
+ }
+ </pre>
+
+ The client's lifecycle should be managed similarly to the following:
+
+ <pre>
+ {@code
+ AMRMClientAsync asyncClient =
+     createAMRMClientAsync(appAttId, 1000, new MyCallbackhandler());
+ asyncClient.init(conf);
+ asyncClient.start();
+ RegisterApplicationMasterResponse response = asyncClient
+    .registerApplicationMaster(appMasterHostname, appMasterRpcPort,
+       appMasterTrackingUrl);
+ asyncClient.addContainerRequest(containerRequest);
+ [... wait for application to complete]
+ asyncClient.unregisterApplicationMaster(status, appMsg, trackingUrl);
+ asyncClient.stop();
+ }
+ </pre>]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.async.AMRMClientAsync -->
+  <!-- start interface org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler -->
+  <interface name="AMRMClientAsync.CallbackHandler"    abstract="true"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="onContainersCompleted"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="statuses" type="java.util.List"/>
+      <doc>
+      <![CDATA[Called when the ResourceManager responds to a heartbeat with completed
+ containers. If the response contains both completed containers and
+ allocated containers, this will be called before containersAllocated.]]>
+      </doc>
+    </method>
+    <method name="onContainersAllocated"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containers" type="java.util.List"/>
+      <doc>
+      <![CDATA[Called when the ResourceManager responds to a heartbeat with allocated
+ containers. If the response containers both completed containers and
+ allocated containers, this will be called after containersCompleted.]]>
+      </doc>
+    </method>
+    <method name="onShutdownRequest"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Called when the ResourceManager wants the ApplicationMaster to shutdown
+ for being out of sync etc. The ApplicationMaster should not unregister
+ with the RM unless the ApplicationMaster wants to be the last attempt.]]>
+      </doc>
+    </method>
+    <method name="onNodesUpdated"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="updatedNodes" type="java.util.List"/>
+      <doc>
+      <![CDATA[Called when nodes tracked by the ResourceManager have changed in health,
+ availability etc.]]>
+      <

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[05/51] [abbrv] hadoop git commit: YARN-5199. Close LogReader in in AHSWebServices#getStreamingOutput and FileInputStream in NMWebServices#getLogs. Contributed by Xuan Gong

Posted by vv...@apache.org.
YARN-5199. Close LogReader in in AHSWebServices#getStreamingOutput and
FileInputStream in NMWebServices#getLogs. Contributed by Xuan Gong


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

Branch: refs/heads/YARN-3926
Commit: 58be55b6e07b94aa55ed87c461f3e5c04cc61630
Parents: 8554aee1b
Author: Xuan <xg...@apache.org>
Authored: Tue Jun 7 16:07:02 2016 -0700
Committer: Xuan <xg...@apache.org>
Committed: Tue Jun 7 16:07:02 2016 -0700

----------------------------------------------------------------------
 .../webapp/AHSWebServices.java                  | 155 ++++++++++---------
 .../nodemanager/webapp/NMWebServices.java       |  71 +++++----
 2 files changed, 118 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/58be55b6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
index d91ae55..59dbd44 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
@@ -40,7 +40,6 @@ import javax.ws.rs.core.Response;
 import javax.ws.rs.core.StreamingOutput;
 import javax.ws.rs.core.Response.ResponseBuilder;
 import javax.ws.rs.core.Response.Status;
-
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
@@ -363,86 +362,94 @@ public class AHSWebServices extends WebServices {
           if ((nodeId == null || nodeName.contains(LogAggregationUtils
               .getNodeString(nodeId))) && !nodeName.endsWith(
               LogAggregationUtils.TMP_FILE_SUFFIX)) {
-            AggregatedLogFormat.LogReader reader =
-                new AggregatedLogFormat.LogReader(conf,
-                    thisNodeFile.getPath());
-            DataInputStream valueStream;
-            LogKey key = new LogKey();
-            valueStream = reader.next(key);
-            while (valueStream != null && !key.toString()
-                .equals(containerIdStr)) {
-              // Next container
-              key = new LogKey();
+            AggregatedLogFormat.LogReader reader = null;
+            try {
+              reader = new AggregatedLogFormat.LogReader(conf,
+                  thisNodeFile.getPath());
+              DataInputStream valueStream;
+              LogKey key = new LogKey();
               valueStream = reader.next(key);
-            }
-            if (valueStream == null) {
-              continue;
-            }
-            while (true) {
-              try {
-                String fileType = valueStream.readUTF();
-                String fileLengthStr = valueStream.readUTF();
-                long fileLength = Long.parseLong(fileLengthStr);
-                if (fileType.equalsIgnoreCase(logFile)) {
-                  StringBuilder sb = new StringBuilder();
-                  sb.append("LogType:");
-                  sb.append(fileType + "\n");
-                  sb.append("Log Upload Time:");
-                  sb.append(Times.format(System.currentTimeMillis()) + "\n");
-                  sb.append("LogLength:");
-                  sb.append(fileLengthStr + "\n");
-                  sb.append("Log Contents:\n");
-                  byte[] b = sb.toString().getBytes(Charset.forName("UTF-8"));
-                  os.write(b, 0, b.length);
-
-                  long toSkip = 0;
-                  long totalBytesToRead = fileLength;
-                  if (bytes < 0) {
-                    long absBytes = Math.abs(bytes);
-                    if (absBytes < fileLength) {
-                      toSkip = fileLength - absBytes;
-                      totalBytesToRead = absBytes;
+              while (valueStream != null && !key.toString()
+                  .equals(containerIdStr)) {
+                // Next container
+                key = new LogKey();
+                valueStream = reader.next(key);
+              }
+              if (valueStream == null) {
+                continue;
+              }
+              while (true) {
+                try {
+                  String fileType = valueStream.readUTF();
+                  String fileLengthStr = valueStream.readUTF();
+                  long fileLength = Long.parseLong(fileLengthStr);
+                  if (fileType.equalsIgnoreCase(logFile)) {
+                    StringBuilder sb = new StringBuilder();
+                    sb.append("LogType:");
+                    sb.append(fileType + "\n");
+                    sb.append("Log Upload Time:");
+                    sb.append(Times.format(System.currentTimeMillis()) + "\n");
+                    sb.append("LogLength:");
+                    sb.append(fileLengthStr + "\n");
+                    sb.append("Log Contents:\n");
+                    byte[] b = sb.toString().getBytes(
+                        Charset.forName("UTF-8"));
+                    os.write(b, 0, b.length);
+
+                    long toSkip = 0;
+                    long totalBytesToRead = fileLength;
+                    if (bytes < 0) {
+                      long absBytes = Math.abs(bytes);
+                      if (absBytes < fileLength) {
+                        toSkip = fileLength - absBytes;
+                        totalBytesToRead = absBytes;
+                      }
+                      long skippedBytes = valueStream.skip(toSkip);
+                      if (skippedBytes != toSkip) {
+                        throw new IOException("The bytes were skipped are "
+                            + "different from the caller requested");
+                      }
+                    } else {
+                      if (bytes < fileLength) {
+                        totalBytesToRead = bytes;
+                      }
                     }
-                    long skippedBytes = valueStream.skip(toSkip);
-                    if (skippedBytes != toSkip) {
-                      throw new IOException("The bytes were skipped are "
-                          + "different from the caller requested");
+
+                    long curRead = 0;
+                    long pendingRead = totalBytesToRead - curRead;
+                    int toRead = pendingRead > buf.length ? buf.length
+                        : (int) pendingRead;
+                    int len = valueStream.read(buf, 0, toRead);
+                    while (len != -1 && curRead < totalBytesToRead) {
+                      os.write(buf, 0, len);
+                      curRead += len;
+
+                      pendingRead = totalBytesToRead - curRead;
+                      toRead = pendingRead > buf.length ? buf.length
+                          : (int) pendingRead;
+                      len = valueStream.read(buf, 0, toRead);
                     }
+                    sb = new StringBuilder();
+                    sb.append("\nEnd of LogType:" + fileType + "\n");
+                    b = sb.toString().getBytes(Charset.forName("UTF-8"));
+                    os.write(b, 0, b.length);
+                    findLogs = true;
                   } else {
-                    if (bytes < fileLength) {
-                      totalBytesToRead = bytes;
+                    long totalSkipped = 0;
+                    long currSkipped = 0;
+                    while (currSkipped != -1 && totalSkipped < fileLength) {
+                      currSkipped = valueStream.skip(
+                          fileLength - totalSkipped);
+                      totalSkipped += currSkipped;
                     }
                   }
-
-                  long curRead = 0;
-                  long pendingRead = totalBytesToRead - curRead;
-                  int toRead = pendingRead > buf.length ? buf.length
-                      : (int) pendingRead;
-                  int len = valueStream.read(buf, 0, toRead);
-                  while (len != -1 && curRead < totalBytesToRead) {
-                    os.write(buf, 0, len);
-                    curRead += len;
-
-                    pendingRead = totalBytesToRead - curRead;
-                    toRead = pendingRead > buf.length ? buf.length
-                        : (int) pendingRead;
-                    len = valueStream.read(buf, 0, toRead);
-                  }
-                  sb = new StringBuilder();
-                  sb.append("\nEnd of LogType:" + fileType + "\n");
-                  b = sb.toString().getBytes(Charset.forName("UTF-8"));
-                  os.write(b, 0, b.length);
-                  findLogs = true;
-                } else {
-                  long totalSkipped = 0;
-                  long currSkipped = 0;
-                  while (currSkipped != -1 && totalSkipped < fileLength) {
-                    currSkipped = valueStream.skip(fileLength - totalSkipped);
-                    totalSkipped += currSkipped;
-                  }
+                } catch (EOFException eof) {
+                  break;
                 }
-              } catch (EOFException eof) {
-                break;
+              }
+            } finally {
+              if (reader != null) {
+                reader.close();
               }
             }
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58be55b6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
index e13baa7..943f3cc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
@@ -37,7 +37,7 @@ import javax.ws.rs.core.Response.ResponseBuilder;
 import javax.ws.rs.core.Response.Status;
 import javax.ws.rs.core.StreamingOutput;
 import javax.ws.rs.core.UriInfo;
-
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -236,7 +236,6 @@ public class NMWebServices {
     }
     boolean downloadFile = parseBooleanParam(download);
     final long bytes = parseLongParam(size);
-
     try {
       final FileInputStream fis = ContainerLogsUtils.openLogFileForRead(
           containerIdStr, logFile, nmContext);
@@ -246,42 +245,46 @@ public class NMWebServices {
         @Override
         public void write(OutputStream os) throws IOException,
             WebApplicationException {
-          int bufferSize = 65536;
-          byte[] buf = new byte[bufferSize];
-          long toSkip = 0;
-          long totalBytesToRead = fileLength;
-          if (bytes < 0) {
-            long absBytes = Math.abs(bytes);
-            if (absBytes < fileLength) {
-              toSkip = fileLength - absBytes;
-              totalBytesToRead = absBytes;
-            }
-            long skippedBytes = fis.skip(toSkip);
-            if (skippedBytes != toSkip) {
-              throw new IOException("The bytes were skipped are different "
-                  + "from the caller requested");
-            }
-          } else {
-            if (bytes < fileLength) {
-              totalBytesToRead = bytes;
+          try {
+            int bufferSize = 65536;
+            byte[] buf = new byte[bufferSize];
+            long toSkip = 0;
+            long totalBytesToRead = fileLength;
+            if (bytes < 0) {
+              long absBytes = Math.abs(bytes);
+              if (absBytes < fileLength) {
+                toSkip = fileLength - absBytes;
+                totalBytesToRead = absBytes;
+              }
+              long skippedBytes = fis.skip(toSkip);
+              if (skippedBytes != toSkip) {
+                throw new IOException("The bytes were skipped are different "
+                    + "from the caller requested");
+              }
+            } else {
+              if (bytes < fileLength) {
+                totalBytesToRead = bytes;
+              }
             }
-          }
-
-          long curRead = 0;
-          long pendingRead = totalBytesToRead - curRead;
-          int toRead = pendingRead > buf.length ? buf.length
-              : (int) pendingRead;
-          int len = fis.read(buf, 0, toRead);
-          while (len != -1 && curRead < totalBytesToRead) {
-            os.write(buf, 0, len);
-            curRead += len;
 
-            pendingRead = totalBytesToRead - curRead;
-            toRead = pendingRead > buf.length ? buf.length
+            long curRead = 0;
+            long pendingRead = totalBytesToRead - curRead;
+            int toRead = pendingRead > buf.length ? buf.length
                 : (int) pendingRead;
-            len = fis.read(buf, 0, toRead);
+            int len = fis.read(buf, 0, toRead);
+            while (len != -1 && curRead < totalBytesToRead) {
+              os.write(buf, 0, len);
+              curRead += len;
+
+              pendingRead = totalBytesToRead - curRead;
+              toRead = pendingRead > buf.length ? buf.length
+                  : (int) pendingRead;
+              len = fis.read(buf, 0, toRead);
+            }
+            os.flush();
+          } finally {
+            IOUtils.closeQuietly(fis);
           }
-          os.flush();
         }
       };
       ResponseBuilder resp = Response.ok(stream);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[46/51] [abbrv] hadoop git commit: YARN-4081. Add support for multiple resource types in the Resource class. (Varun Vasudev via wangda)

Posted by vv...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e518ef1/hadoop-build-tools/src/main/resources/META-INF/LICENSE.txt
----------------------------------------------------------------------
diff --git a/hadoop-build-tools/src/main/resources/META-INF/LICENSE.txt b/hadoop-build-tools/src/main/resources/META-INF/LICENSE.txt
new file mode 100644
index 0000000..44880df
--- /dev/null
+++ b/hadoop-build-tools/src/main/resources/META-INF/LICENSE.txt
@@ -0,0 +1,1661 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+
+
+APACHE HADOOP SUBCOMPONENTS:
+
+The Apache Hadoop project contains subcomponents with separate copyright
+notices and license terms. Your use of the source code for the these
+subcomponents is subject to the terms and conditions of the following
+licenses. 
+
+For the org.apache.hadoop.util.bloom.* classes:
+
+/**
+ *
+ * Copyright (c) 2005, European Commission project OneLab under contract
+ * 034819 (http://www.one-lab.org)
+ * All rights reserved.
+ * Redistribution and use in source and binary forms, with or 
+ * without modification, are permitted provided that the following 
+ * conditions are met:
+ *  - Redistributions of source code must retain the above copyright 
+ *    notice, this list of conditions and the following disclaimer.
+ *  - Redistributions in binary form must reproduce the above copyright 
+ *    notice, this list of conditions and the following disclaimer in 
+ *    the documentation and/or other materials provided with the distribution.
+ *  - Neither the name of the University Catholique de Louvain - UCL
+ *    nor the names of its contributors may be used to endorse or 
+ *    promote products derived from this software without specific prior 
+ *    written permission.
+ *    
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 
+ * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS 
+ * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE 
+ * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, 
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, 
+ * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; 
+ * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER 
+ * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT 
+ * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN 
+ * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE 
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+
+For portions of the native implementation of slicing-by-8 CRC calculation
+in src/main/native/src/org/apache/hadoop/util:
+
+/**
+ *   Copyright 2008,2009,2010 Massachusetts Institute of Technology.
+ *   All rights reserved. Use of this source code is governed by a
+ *   BSD-style license that can be found in the LICENSE file.
+ */
+
+For src/main/native/src/org/apache/hadoop/io/compress/lz4/{lz4.h,lz4.c,lz4hc.h,lz4hc.c},
+
+/*
+   LZ4 - Fast LZ compression algorithm
+   Header File
+   Copyright (C) 2011-2014, Yann Collet.
+   BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php)
+
+   Redistribution and use in source and binary forms, with or without
+   modification, are permitted provided that the following conditions are
+   met:
+
+       * Redistributions of source code must retain the above copyright
+   notice, this list of conditions and the following disclaimer.
+       * Redistributions in binary form must reproduce the above
+   copyright notice, this list of conditions and the following disclaimer
+   in the documentation and/or other materials provided with the
+   distribution.
+
+   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+   You can contact the author at :
+   - LZ4 source repository : http://code.google.com/p/lz4/
+   - LZ4 public forum : https://groups.google.com/forum/#!forum/lz4c
+*/
+
+
+For hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/native/gtest
+---------------------------------------------------------------------
+Copyright 2008, Google Inc.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+    * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+re2j 1.0
+---------------------------------------------------------------------
+This is a work derived from Russ Cox's RE2 in Go, whose license
+http://golang.org/LICENSE is as follows:
+
+Copyright (c) 2009 The Go Authors. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+   * Redistributions of source code must retain the above copyright
+     notice, this list of conditions and the following disclaimer.
+
+   * Redistributions in binary form must reproduce the above copyright
+     notice, this list of conditions and the following disclaimer in
+     the documentation and/or other materials provided with the
+     distribution.
+
+   * Neither the name of Google Inc. nor the names of its contributors
+     may be used to endorse or promote products derived from this
+     software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+For hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/util/tree.h
+---------------------------------------------------------------------
+Copyright 2002 Niels Provos <pr...@citi.umich.edu>
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions
+are met:
+1. Redistributions of source code must retain the above copyright
+   notice, this list of conditions and the following disclaimer.
+2. Redistributions in binary form must reproduce the above copyright
+   notice, this list of conditions and the following disclaimer in the
+   documentation and/or other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
+IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+The binary distribution of this product bundles binaries of leveldbjni
+(https://github.com/fusesource/leveldbjni), which is available under the
+following license:
+
+Copyright (c) 2011 FuseSource Corp. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+   * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+   * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+   * Neither the name of FuseSource Corp. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+The binary distribution of this product bundles binaries of leveldb
+(http://code.google.com/p/leveldb/), which is available under the following
+license:
+
+Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+   * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+   * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+   * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+The binary distribution of this product bundles binaries of snappy
+(http://code.google.com/p/snappy/), which is available under the following
+license:
+
+Copyright 2011, Google Inc.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+    * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+For:
+hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dataTables.bootstrap.js
+hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dataTables.bootstrap.css
+hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/jquery.dataTables.min.js
+hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/
+--------------------------------------------------------------------------------
+Copyright (C) 2008-2016, SpryMedia Ltd.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+For:
+hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-full-2.0.0.min.js
+hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-helpers-1.1.1.min.js
+--------------------------------------------------------------------------------
+
+Copyright (c) 2010 Aleksander Williams
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+For:
+hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/moment.min.js
+--------------------------------------------------------------------------------
+
+Copyright (c) 2011-2016 Tim Wood, Iskren Chernev, Moment.js contributors
+
+Permission is hereby granted, free of charge, to any person
+obtaining a copy of this software and associated documentation
+files (the "Software"), to deal in the Software without
+restriction, including without limitation the rights to use,
+copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the
+Software is furnished to do so, subject to the following
+conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
+OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
+HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
+WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+OTHER DEALINGS IN THE SOFTWARE.
+
+For:
+hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2
+hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/bootstrap.min.js
+hadoop-tools/hadoop-sls/src/main/html/css/bootstrap.min.css
+hadoop-tools/hadoop-sls/src/main/html/css/bootstrap-responsive.min.css
+And the binary distribution of this product bundles these dependencies under the
+following license:
+Mockito 1.8.5
+SLF4J 1.7.10
+--------------------------------------------------------------------------------
+
+The MIT License (MIT)
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+For:
+hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/jquery-1.10.2.min.js
+hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/jquery.js
+hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery
+--------------------------------------------------------------------------------
+
+Copyright jQuery Foundation and other contributors, https://jquery.org/
+
+This software consists of voluntary contributions made by many
+individuals. For exact contribution history, see the revision history
+available at https://github.com/jquery/jquery
+
+The following license applies to all parts of this software except as
+documented below:
+
+====
+
+Permission is hereby granted, free of charge, to any person obtaining
+a copy of this software and associated documentation files (the
+"Software"), to deal in the Software without restriction, including
+without limitation the rights to use, copy, modify, merge, publish,
+distribute, sublicense, and/or sell copies of the Software, and to
+permit persons to whom the Software is furnished to do so, subject to
+the following conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+====
+
+All files located in the node_modules and external directories are
+externally maintained libraries used by this software which have their
+own licenses; we recommend you read them, as their terms may differ from
+the terms above.
+
+For:
+hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jt/jquery.jstree.js.gz
+--------------------------------------------------------------------------------
+
+Copyright (c) 2014 Ivan Bozhanov
+
+Permission is hereby granted, free of charge, to any person
+obtaining a copy of this software and associated documentation
+files (the "Software"), to deal in the Software without
+restriction, including without limitation the rights to use,
+copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the
+Software is furnished to do so, subject to the following
+conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
+OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
+HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
+WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+OTHER DEALINGS IN THE SOFTWARE.
+
+For:
+hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3.v3.js
+--------------------------------------------------------------------------------
+
+D3 is available under a 3-clause BSD license. For details, see:
+hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3-LICENSE
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+HSQLDB Database 2.0.0
+--------------------------------------------------------------------------------
+"COPYRIGHTS AND LICENSES (based on BSD License)
+
+For work developed by the HSQL Development Group:
+
+Copyright (c) 2001-2016, The HSQL Development Group
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice, this
+list of conditions and the following disclaimer.
+
+Redistributions in binary form must reproduce the above copyright notice,
+this list of conditions and the following disclaimer in the documentation
+and/or other materials provided with the distribution.
+
+Neither the name of the HSQL Development Group nor the names of its
+contributors may be used to endorse or promote products derived from this
+software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS""
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ARE DISCLAIMED. IN NO EVENT SHALL HSQL DEVELOPMENT GROUP, HSQLDB.ORG,
+OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+For work originally developed by the Hypersonic SQL Group:
+
+Copyright (c) 1995-2000 by the Hypersonic SQL Group.
+All rights reserved.
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice, this
+list of conditions and the following disclaimer.
+
+Redistributions in binary form must reproduce the above copyright notice,
+this list of conditions and the following disclaimer in the documentation
+and/or other materials provided with the distribution.
+
+Neither the name of the Hypersonic SQL Group nor the names of its
+contributors may be used to endorse or promote products derived from this
+software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS""
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ARE DISCLAIMED. IN NO EVENT SHALL THE HYPERSONIC SQL GROUP,
+OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+This software consists of voluntary contributions made by many individuals on behalf of the
+Hypersonic SQL Group."
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+servlet-api 2.5
+jsp-api 2.1
+Streaming API for XML 1.0
+--------------------------------------------------------------------------------
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0
+1.�Definitions.�
+
+1.1.�Contributor�means each individual or entity
+that creates or contributes to the creation of
+Modifications.�
+
+1.2.�Contributor Version�means the combination of the
+Original Software, prior Modifications used by a Contributor (if any), and the
+Modifications made by that particular Contributor.�
+
+1.3.�Covered
+Software�means (a) the Original Software, or (b) Modifications, or (c) the
+combination of files containing Original Software with files containing
+Modifications, in each case including portions
+thereof.�
+
+1.4.�Executable�means the Covered Software in any form other
+than Source Code.�
+
+1.5.�Initial Developer�means the individual or entity
+that first makes Original Software available under this
+License.�
+
+1.6.�Larger Work�means a work which combines Covered Software or
+portions thereof with code not governed by the terms of this
+License.�
+
+1.7.�License�means this document.�
+
+1.8.�Licensable�means
+having the right to grant, to the maximum extent possible, whether at the time
+of the initial grant or subsequently acquired, any and all of the rights
+conveyed herein.�
+
+1.9.�Modifications�means the Source Code and Executable
+form of any of the following:
+A. Any file that results from an addition to,
+deletion from or modification of the contents of a file containing Original
+Software or previous Modifications;
+B. Any new file that contains any part of the Original Software
+or previous Modification; or
+C. Any new file that is contributed or otherwise made available
+under the terms of this License.�
+
+1.10.�Original Software�means the Source Code and Executable form of
+computer software code that is originally released under this License.�
+
+1.11.�Patent Claims�means any patent claim(s), now owned or
+hereafter acquired, including without limitation, method, process, and apparatus
+claims, in any patent Licensable by grantor.�
+
+1.12.�Source Code�means (a) the common form of computer software code in which
+modifications are made and (b) associated documentation included in or
+with such code.�
+
+1.13.�You (or Your)�means an individual or a legal entity exercising rights
+under, and complying with all of the terms of, this License. For legal entities,
+You includes any entity which controls, is controlled by, or is under common control
+with You. For purposes of this definition, control means (a)�the power, direct
+or indirect, to cause the direction or management of such entity, whether by
+contract or otherwise, or (b)�ownership of more than fifty percent (50%) of the
+outstanding shares or beneficial ownership of such entity.�
+
+2. License Grants.
+
+2.1. The Initial Developer Grant. Conditioned upon Your compliance
+with Section 3.1 below and subject to third party intellectual property claims,
+the Initial Developer hereby grants You a world-wide, royalty-free,
+non-exclusive license:�
+
+(a) under intellectual property rights (other than
+patent or trademark) Licensable by Initial Developer, to use, reproduce, modify,
+display, perform, sublicense and distribute the Original Software (or portions
+thereof), with or without Modifications, and/or as part of a Larger Work;
+and�
+
+(b) under Patent Claims infringed by the making, using or selling of
+Original Software, to make, have made, use, practice, sell, and offer for sale,
+and/or otherwise dispose of the Original Software (or portions
+thereof);
+
+(c) The licenses granted in Sections�2.1(a) and (b) are
+effective on the date Initial Developer first distributes or otherwise makes the
+Original Software available to a third party under the terms of this
+License;
+
+(d) Notwithstanding Section�2.1(b) above, no patent license is
+granted: (1)�for code that You delete from the Original Software, or (2)�for
+infringements caused by: (i)�the modification of the Original Software, or
+(ii)�the combination of the Original Software with other software or
+devices.�
+
+2.2. Contributor Grant. Conditioned upon Your compliance with
+Section 3.1 below and subject to third party intellectual property claims, each
+Contributor hereby grants You a world-wide, royalty-free, non-exclusive
+license:�
+
+(a) under intellectual property rights (other than patent or
+trademark) Licensable by Contributor to use, reproduce, modify, display,
+perform, sublicense and distribute the Modifications created by such Contributor
+(or portions thereof), either on an unmodified basis, with other Modifications,
+as Covered Software and/or as part of a Larger Work; and�
+
+(b) under Patent
+Claims infringed by the making, using, or selling of Modifications made by that
+Contributor either alone and/or in combination with its Contributor Version (or
+portions of such combination), to make, use, sell, offer for sale, have made,
+and/or otherwise dispose of: (1)�Modifications made by that Contributor (or
+portions thereof); and (2)�the combination of Modifications made by that
+Contributor with its Contributor Version (or portions of such
+combination).�
+
+(c) The licenses granted in Sections�2.2(a) and 2.2(b) are
+effective on the date Contributor first distributes or otherwise makes the
+Modifications available to a third party.
+
+(d) Notwithstanding Section�2.2(b)
+above, no patent license is granted: (1)�for any code that Contributor has
+deleted from the Contributor Version; (2)�for infringements caused by:
+(i)�third party modifications of Contributor Version, or (ii)�the combination
+of Modifications made by that Contributor with other software (except as part of
+the Contributor Version) or other devices; or (3)�under Patent Claims infringed
+by Covered Software in the absence of Modifications made by that
+Contributor.�
+
+3. Distribution Obligations.�
+
+3.1. Availability of Source
+Code. Any Covered Software that You distribute or otherwise make available in
+Executable form must also be made available in Source Code form and that Source
+Code form must be distributed only under the terms of this License. You must
+include a copy of this License with every copy of the Source Code form of the
+Covered Software You distribute or otherwise make available. You must inform
+recipients of any such Covered Software in Executable form as to how they can
+obtain such Covered Software in Source Code form in a reasonable manner on or
+through a medium customarily used for software exchange.�
+
+3.2.
+Modifications. The Modifications that You create or to which You contribute are
+governed by the terms of this License. You represent that You believe Your
+Modifications are Your original creation(s) and/or You have sufficient rights to
+grant the rights conveyed by this License.�
+
+3.3. Required Notices. You must
+include a notice in each of Your Modifications that identifies You as the
+Contributor of the Modification. You may not remove or alter any copyright,
+patent or trademark notices contained within the Covered Software, or any
+notices of licensing or any descriptive text giving attribution to any
+Contributor or the Initial Developer.�
+
+3.4. Application of Additional Terms.
+You may not offer or impose any terms on any Covered Software in Source Code
+form that alters or restricts the applicable version of this License or the
+recipients rights hereunder. You may choose to offer, and to charge a fee for,
+warranty, support, indemnity or liability obligations to one or more recipients
+of Covered Software. However, you may do so only on Your own behalf, and not on
+behalf of the Initial Developer or any Contributor. You must make it absolutely
+clear that any such warranty, support, indemnity or liability obligation is
+offered by You alone, and You hereby agree to indemnify the Initial Developer
+and every Contributor for any liability incurred by the Initial Developer or
+such Contributor as a result of warranty, support, indemnity or liability terms
+You offer.
+
+3.5. Distribution of Executable Versions. You may distribute the
+Executable form of the Covered Software under the terms of this License or under
+the terms of a license of Your choice, which may contain terms different from
+this License, provided that You are in compliance with the terms of this License
+and that the license for the Executable form does not attempt to limit or alter
+the recipients rights in the Source Code form from the rights set forth in this
+License. If You distribute the Covered Software in Executable form under a
+different license, You must make it absolutely clear that any terms which differ
+from this License are offered by You alone, not by the Initial Developer or
+Contributor. You hereby agree to indemnify the Initial Developer and every
+Contributor for any liability incurred by the Initial Developer or such
+Contributor as a result of any such terms You offer.�
+
+3.6. Larger Works. You
+may create a Larger Work by combining Covered Software with other code not
+governed by the terms of this License and distribute the Larger Work as a single
+product. In such a case, You must make sure the requirements of this License are
+fulfilled for the Covered Software.�
+
+4. Versions of the License.�
+
+4.1.
+New Versions. Sun Microsystems, Inc. is the initial license steward and may
+publish revised and/or new versions of this License from time to time. Each
+version will be given a distinguishing version number. Except as provided in
+Section 4.3, no one other than the license steward has the right to modify this
+License.�
+
+4.2. Effect of New Versions. You may always continue to use,
+distribute or otherwise make the Covered Software available under the terms of
+the version of the License under which You originally received the Covered
+Software. If the Initial Developer includes a notice in the Original Software
+prohibiting it from being distributed or otherwise made available under any
+subsequent version of the License, You must distribute and make the Covered
+Software available under the terms of the version of the License under which You
+originally received the Covered Software. Otherwise, You may also choose to use,
+distribute or otherwise make the Covered Software available under the terms of
+any subsequent version of the License published by the license
+steward.�
+
+4.3. Modified Versions. When You are an Initial Developer and You
+want to create a new license for Your Original Software, You may create and use
+a modified version of this License if You: (a)�rename the license and remove
+any references to the name of the license steward (except to note that the
+license differs from this License); and (b)�otherwise make it clear that the
+license contains terms which differ from this License.�
+
+5. DISCLAIMER OF WARRANTY.
+
+COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN AS IS BASIS,
+WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT
+LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS,
+MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS
+TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY
+COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER
+OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR
+CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS
+LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER
+THIS DISCLAIMER.�
+
+6. TERMINATION.�
+
+6.1. This License and the rights
+granted hereunder will terminate automatically if You fail to comply with terms
+herein and fail to cure such breach within 30 days of becoming aware of the
+breach. Provisions which, by their nature, must remain in effect beyond the
+termination of this License shall survive.�
+
+6.2. If You assert a patent
+infringement claim (excluding declaratory judgment actions) against Initial
+Developer or a Contributor (the Initial Developer or Contributor against whom
+You assert such claim is referred to as Participant) alleging that the
+Participant Software (meaning the Contributor Version where the Participant is a
+Contributor or the Original Software where the Participant is the Initial
+Developer) directly or indirectly infringes any patent, then any and all rights
+granted directly or indirectly to You by such Participant, the Initial Developer
+(if the Initial Developer is not the Participant) and all Contributors under
+Sections�2.1 and/or 2.2 of this License shall, upon 60 days notice from
+Participant terminate prospectively and automatically at the expiration of such
+60 day notice period, unless if within such 60 day period You withdraw Your
+claim with respect to the Participant Software against such Participant either
+unilaterally or pursuant to a written agreement with Participant.�
+
+6.3. In
+the event of termination under Sections�6.1 or 6.2 above, all end user licenses
+that have been validly granted by You or any distributor hereunder prior to
+termination (excluding licenses granted to You by any distributor) shall survive
+termination.�
+
+7. LIMITATION OF LIABILITY.
+UNDER NO CIRCUMSTANCES AND UNDER
+NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE,
+SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF
+COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY
+PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY
+CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF
+GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER
+COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE
+POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO
+LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTYS NEGLIGENCE TO
+THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT
+ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO
+THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU.�
+
+8. U.S. GOVERNMENT END USERS.
+
+The Covered Software is a commercial item, as that term is defined in
+48�C.F.R.�2.101 (Oct. 1995), consisting of commercial computer software (as
+that term is defined at 48 C.F.R. �252.227-7014(a)(1)) and commercial computer
+software documentation as such terms are used in 48�C.F.R.�12.212 (Sept.
+1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through
+227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software
+with only those rights set forth herein. This U.S. Government Rights clause is
+in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision
+that addresses Government rights in computer software under this
+License.�
+
+9. MISCELLANEOUS.
+This License represents the complete agreement
+concerning subject matter hereof. If any provision of this License is held to be
+unenforceable, such provision shall be reformed only to the extent necessary to
+make it enforceable. This License shall be governed by the law of the
+jurisdiction specified in a notice contained within the Original Software
+(except to the extent applicable law, if any, provides otherwise), excluding
+such jurisdictions conflict-of-law provisions. Any litigation relating to this
+License shall be subject to the jurisdiction of the courts located in the
+jurisdiction and venue specified in a notice contained within the Original
+Software, with the losing party responsible for costs, including, without
+limitation, court costs and reasonable attorneys fees and expenses. The
+application of the United Nations Convention on Contracts for the International
+Sale of Goods is expressly excluded. Any law or regulation which provides that
+the language of a contract shall be construed against the drafter shall not
+apply to this License. You agree that You alone are responsible for compliance
+with the United States export administration regulations (and the export control
+laws and regulation of any other countries) when You use, distribute or
+otherwise make available any Covered Software.�
+
+10. RESPONSIBILITY FOR CLAIMS.
+As between Initial Developer and the Contributors, each party is
+responsible for claims and damages arising, directly or indirectly, out of its
+utilization of rights under this License and You agree to work with Initial
+Developer and Contributors to distribute such responsibility on an equitable
+basis. Nothing herein is intended or shall be deemed to constitute any admission
+of liability.�
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+Jersey 1.9
+JAXB API bundle for GlassFish V3 2.2.2
+JAXB RI 2.2.3
+--------------------------------------------------------------------------------
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)Version 1.1
+
+1. Definitions.
+
+1.1. \u201cContributor\u201d means each individual or entity that creates or
+contributes to the creation of Modifications.
+1.2. \u201cContributor Version\u201d means the combination of the Original Software,
+prior Modifications used by a Contributor (if any), and the Modifications made
+by that particular Contributor.
+1.3. \u201cCovered Software\u201d means (a) the Original Software, or (b)
+Modifications, or (c) the combination of files containing Original Software with
+files containing Modifications, in each case including portions thereof.
+1.4. \u201cExecutable\u201d means the Covered Software in any form other than Source
+Code.
+1.5. \u201cInitial Developer\u201d means the individual or entity that first makes
+Original Software available under this License.
+1.6. \u201cLarger Work\u201d means a work which combines Covered Software or portions
+thereof with code not governed by the terms of this License.
+1.7. \u201cLicense\u201d means this document.
+1.8. \u201cLicensable\u201d means having the right to grant, to the maximum extent
+possible, whether at the time of the initial grant or subsequently acquired, any
+and all of the rights conveyed herein.
+1.9. \u201cModifications\u201d means the Source Code and Executable form of any of the
+following:
+A. Any file that results from an addition to, deletion from or modification of
+the contents of a file containing Original Software or previous Modifications;
+B. Any new file that contains any part of the Original Software or previous
+Modification; or
+C. Any new file that is contributed or otherwise made available under the terms
+of this License.
+1.10. \u201cOriginal Software\u201d means the Source Code and Executable form of
+computer software code that is originally released under this License.
+1.11. \u201cPatent Claims\u201d means any patent claim(s), now owned or hereafter
+acquired, including without limitation, method, process, and apparatus claims,
+in any patent Licensable by grantor.
+1.12. \u201cSource Code\u201d means (a) the common form of computer software code in
+which modifications are made and (b) associated documentation included in or
+with such code.
+1.13. \u201cYou\u201d (or \u201cYour\u201d) means an individual or a legal entity exercising
+rights under, and complying with all of the terms of, this License. For legal
+entities, \u201cYou\u201d includes any entity which controls, is controlled by, or is
+under common control with You. For purposes of this definition, \u201ccontrol\u201d
+means (a) the power, direct or indirect, to cause the direction or management of
+such entity, whether by contract or otherwise, or (b) ownership of more than
+fifty percent (50%) of the outstanding shares or beneficial ownership of such
+entity.
+
+2. License Grants.
+
+2.1. The Initial Developer Grant.
+
+Conditioned upon Your compliance with Section 3.1 below and subject to
+third party intellectual property claims, the Initial Developer hereby grants
+You a world-wide, royalty-free, non-exclusive license:
+(a) under intellectual
+property rights (other than patent or trademark) Licensable by Initial
+Developer, to use, reproduce, modify, display, perform, sublicense and
+distribute the Original Software (or portions thereof), with or without
+Modifications, and/or as part of a Larger Work; and
+(b) under Patent Claims
+infringed by the making, using or selling of Original Software, to make, have
+made, use, practice, sell, and offer for sale, and/or otherwise dispose of the
+Original Software (or portions thereof).
+(c) The licenses granted in Sections
+2.1(a) and (b) are effective on the date Initial Developer first distributes or
+otherwise makes the Original Software available to a third party under the terms
+of this License.
+(d) Notwithstanding Section 2.1(b) above, no patent license is
+granted: (1) for code that You delete from the Original Software, or (2) for
+infringements caused by: (i) the modification of the Original Software, or (ii)
+the combination of the Original Software with other software or devices.
+
+2.2. Contributor Grant.
+
+Conditioned upon Your compliance with Section 3.1 below and
+subject to third party intellectual property claims, each Contributor hereby
+grants You a world-wide, royalty-free, non-exclusive license:
+(a) under
+intellectual property rights (other than patent or trademark) Licensable by
+Contributor to use, reproduce, modify, display, perform, sublicense and
+distribute the Modifications created by such Contributor (or portions thereof),
+either on an unmodified basis, with other Modifications, as Covered Software
+and/or as part of a Larger Work; and
+(b) under Patent Claims infringed by the
+making, using, or selling of Modifications made by that Contributor either alone
+and/or in combination with its Contributor Version (or portions of such
+combination), to make, use, sell, offer for sale, have made, and/or otherwise
+dispose of: (1) Modifications made by that Contributor (or portions thereof);
+and (2) the combination of Modifications made by that Contributor with its
+Contributor Version (or portions of such combination).
+(c) The licenses granted
+in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first
+distributes or otherwise makes the Modifications available to a third
+party.
+(d) Notwithstanding Section 2.2(b) above, no patent license is granted:
+(1) for any code that Contributor has deleted from the Contributor Version; (2)
+for infringements caused by: (i) third party modifications of Contributor
+Version, or (ii) the combination of Modifications made by that Contributor with
+other software (except as part of the Contributor Version) or other devices; or
+(3) under Patent Claims infringed by Covered Software in the absence of
+Modifications made by that Contributor.
+
+3. Distribution Obligations.
+
+3.1. Availability of Source Code.
+Any Covered Software that You distribute or
+otherwise make available in Executable form must also be made available in
+Source Code form and that Source Code form must be distributed only under the
+terms of this License. You must include a copy of this License with every copy
+of the Source Code form of the Covered Software You distribute or otherwise make
+available. You must inform recipients of any such Covered Software in Executable
+form as to how they can obtain such Covered Software in Source Code form in a
+reasonable manner on or through a medium customarily used for software
+exchange.
+3.2. Modifications.
+The Modifications that You create or to which
+You contribute are governed by the terms of this License. You represent that You
+believe Your Modifications are Your original creation(s) and/or You have
+sufficient rights to grant the rights conveyed by this License.
+3.3. Required Notices.
+You must include a notice in each of Your Modifications that
+identifies You as the Contributor of the Modification. You may not remove or
+alter any copyright, patent or trademark notices contained within the Covered
+Software, or any notices of licensing or any descriptive text giving attribution
+to any Contributor or the Initial Developer.
+3.4. Application of Additional Terms.
+You may not offer or impose any terms on any Covered Software in Source
+Code form that alters or restricts the applicable version of this License or the
+recipients' rights hereunder. You may choose to offer, and to charge a fee for,
+warranty, support, indemnity or liability obligations to one or more recipients
+of Covered Software. However, you may do so only on Your own behalf, and not on
+behalf of the Initial Developer or any Contributor. You must make it absolutely
+clear that any such warranty, support, indemnity or liability obligation is
+offered by You alone, and You hereby agree to indemnify the Initial Developer
+and every Contributor for any liability incurred by the Initial Developer or
+such Contributor as a result of warranty, support, indemnity or liability terms
+You offer.
+3.5. Distribution of Executable Versions.
+You may distribute the
+Executable form of the Covered Software under the terms of this License or under
+the terms of a license of Your choice, which may contain terms different from
+this License, provided that You are in compliance with the terms of this License
+and that the license for the Executable form does not attempt to limit or alter
+the recipient's rights in the Source Code form from the rights set forth in
+this License. If You distribute the Covered Software in Executable form under a
+different license, You must make it absolutely clear that any terms which differ
+from this License are offered by You alone, not by the Initial Developer or
+Contributor. You hereby agree to indemnify the Initial Developer and every
+Contributor for any liability incurred by the Initial Developer or such
+Contributor as a result of any such terms You offer.
+3.6. Larger Works.
+You
+may create a Larger Work by combining Covered Software with other code not
+governed by the terms of this License and distribute the Larger Work as a single
+product. In such a case, You must make sure the requirements of this License are
+fulfilled for the Covered Software.
+
+4. Versions of the License.
+
+4.1. New Versions.
+Oracle is the initial license steward and may publish revised and/or
+new versions of this License from time to time. Each version will be given a
+distinguishing version number. Except as provided in Section 4.3, no one other
+than the license steward has the right to modify this License.
+4.2. Effect of New Versions.
+You may always continue to use, distribute or otherwise make the
+Covered Software available under the terms of the version of the License under
+which You originally received the Covered Software. If the Initial Developer
+includes a notice in the Original Software prohibiting it from being distributed
+or otherwise made available under any subsequent version of the License, You
+must distribute and make the Covered Software available under the terms of the
+version of the License under which You originally received the Covered Software.
+Otherwise, You may also choose to use, distribute or otherwise make the Covered
+Software available under the terms of any subsequent version of the License
+published by the license steward.
+4.3. Modified Versions.
+When You are an
+Initial Developer and You want to create a new license for Your Original
+Software, You may create and use a modified version of this License if You: (a)
+rename the license and remove any references to the name of the license steward
+(except to note that the license differs from this License); and (b) otherwise
+make it clear that the license contains terms which differ from this
+License.
+
+5. DISCLAIMER OF WARRANTY.
+
+COVERED SOFTWARE IS PROVIDED UNDER THIS
+LICENSE ON AN \u201cAS IS\u201d BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED
+OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE
+IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR
+NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED
+SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY
+RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE
+COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF
+WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED
+SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER.
+
+6. TERMINATION.
+
+6.1. This License and the rights granted hereunder will
+terminate automatically if You fail to comply with terms herein and fail to cure
+such breach within 30 days of becoming aware of the breach. Provisions which, by
+their nature, must remain in effect beyond the termination of this License shall
+survive.
+6.2. If You assert a patent infringement claim (excluding declaratory
+judgment actions) against Initial Developer or a Contributor (the Initial
+Developer or Contributor against whom You assert such claim is referred to as
+\u201cParticipant\u201d) alleging that the Participant Software (meaning the
+Contributor Version where the Participant is a Contributor or the Original
+Software where the Participant is the Initial Developer) directly or indirectly
+infringes any patent, then any and all rights granted directly or indirectly to
+You by such Participant, the Initial Developer (if the Initial Developer is not
+the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this
+License shall, upon 60 days notice from Participant terminate prospectively and
+automatically at the expiration of such 60 day notice period, unless if within
+such 60 day period You withdraw Your claim with respect to the Participant
+Software against such Participant either unilaterally or pursuant to a written
+agreement with Participant.
+6.3. If You assert a patent infringement claim
+against Participant alleging that the Participant Software directly or
+indirectly infringes any patent where such claim is resolved (such as by license
+or settlement) prior to the initiation of patent infringement litigation, then
+the reasonable value of the licenses granted by such Participant under Sections
+2.1 or 2.2 shall be taken into account in determining the amount or value of any
+payment or license.
+6.4. In the event of termination under Sections 6.1 or 6.2
+above, all end user licenses that have been validly granted by You or any
+distributor hereunder prior to termination (excluding licenses granted to You by
+any distributor) shall survive termination.
+
+7. LIMITATION OF LIABILITY.
+
+UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT
+(INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL
+DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY
+SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT,
+SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING,
+WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER
+FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN
+IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS
+LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL
+INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW
+PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR
+LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND
+LIMITATION MAY NOT APPLY TO YOU.
+
+8. U.S. GOVERNMENT END USERS.
+
+The Covered
+Software is a \u201ccommercial item,\u201d as that term is defined in 48 C.F.R. 2.101
+(Oct. 1995), consisting of \u201ccommercial computer software\u201d (as that term is
+defined at 48 C.F.R. � 252.227-7014(a)(1)) and \u201ccommercial computer software
+documentation\u201d as such terms are used in 48 C.F.R. 12.212 (Sept. 1995).
+Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4
+(June 1995), all U.S. Government End Users acquire Covered Software with only
+those rights set forth herein. This U.S. Government Rights clause is in lieu of,
+and supersedes, any other FAR, DFAR, or other clause or provision that addresses
+Government rights in computer software under this License.
+
+9. MISCELLANEOUS.
+
+This License represents the complete agreement concerning
+subject matter hereof. If any provision of this License is held to be
+unenforceable, such provision shall be reformed only to the extent necessary to
+make it enforceable. This License shall be governed by the law of the
+jurisdiction specified in a notice contained within the Original Software
+(except to the extent applicable law, if any, provides otherwise), excluding
+such jurisdiction's conflict-of-law provisions. Any litigation relating to this
+License shall be subject to the jurisdiction of the courts located in the
+jurisdiction and venue specified in a notice contained within the Original
+Software, with the losing party responsible for costs, including, without
+limitation, court costs and reasonable attorneys' fees and expenses. The
+application of the United Nations Convention on Contracts for the International
+Sale of Goods is expressly excluded. Any law or regulation which provides that
+the language of a contract shall be construed against the drafter shall not
+apply to this License. You agree that You alone are responsible for compliance
+with the United States export administration regulations (and the export control
+laws and regulation of any other countries) when You use, distribute or
+otherwise make available any Covered Software.
+
+10. RESPONSIBILITY FOR CLAIMS.
+
+As between Initial Developer and the Contributors, each party is
+responsible for claims and damages arising, directly or indirectly, out of its
+utilization of rights under this License and You agree to work with Initial
+Developer and Contributors to distribute such responsibility on an equitable
+basis. Nothing herein is intended or shall be deemed to constitute any admission
+of liability.
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+Protocol Buffer Java API 2.5.0
+--------------------------------------------------------------------------------
+This license applies to all parts of Protocol Buffers except the following:
+
+  - Atomicops support for generic gcc, located in
+    src/google/protobuf/stubs/atomicops_internals_generic_gcc.h.
+    This file is copyrighted by Red Hat Inc.
+
+  - Atomicops support for AIX/POWER, located in
+    src/google/protobuf/stubs/atomicops_internals_power.h.
+    This file is copyrighted by Bloomberg Finance LP.
+
+Copyright 2014, Google Inc.  All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+    * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+Code generated by the Protocol Buffer compiler is owned by the owner
+of the input file used when generating it.  This code is not
+standalone and requires a support library to be linked with it.  This
+support library is itself covered by the above license.
+
+For:
+XML Commons External Components XML APIs 1.3.04
+--------------------------------------------------------------------------------
+By obtaining, using and/or copying this work, you (the licensee) agree that you
+have read, understood, and will comply with the following terms and conditions.
+
+Permission to copy, modify, and distribute this software and its documentation,
+with or without modification, for any purpose and without fee or royalty is
+hereby granted, provided that you include the following on ALL copies of the
+software and documentation or portions thereof, including modifications:
+- The full text of this NOTICE in a location viewable to users of the
+redistributed or derivative work.
+- Any pre-existing intellectual property disclaimers, notices, or terms and
+conditions. If none exist, the W3C Software Short Notice should be included
+(hypertext is preferred, text is permitted) within the body of any redistributed
+or derivative code.
+- Notice of any changes or modifications to the files, including the date changes
+were made. (We recommend you provide URIs to the location from which the code is
+derived.)
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+JUnit 4.11
+ecj-4.3.1.jar
+--------------------------------------------------------------------------------
+Eclipse Public License - v 1.0
+
+THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC
+LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM
+CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
+
+1. DEFINITIONS
+
+"Contribution" means:
+
+a) in the case of the initial Contributor, the initial code and documentation
+distributed under this Agreement, and
+b) in the case of each subsequent Contributor:
+i) changes to the Program, and
+ii) additions to the Program;
+where such changes and/or additions to the Program originate from and are
+distributed by that particular Contributor. A Contribution 'originates' from a
+Contributor if it was added to the Program by such Contributor itself or anyone
+acting on such Contributor's behalf. Contributions do not include additions to
+the Program which: (i) are separate modules of software distributed in
+conjunction with the Program under their own license agreement, and (ii) are not
+derivative works of the Program.
+"Contributor" means any person or entity that distributes the Program.
+
+"Licensed Patents" mean patent claims licensable by a Contributor which are
+necessarily infringed by the use or sale of its Contribution alone or when
+combined with the Program.
+
+"Program" means the Contributions distributed in accordance with this Agreement.
+
+"Recipient" means anyone who receives the Program under this Agreement,
+including all Contributors.
+
+2. GRANT OF RIGHTS
+
+a) Subject to the terms of this Agreement, each Contributor hereby grants
+Recipient a non-exclusive, worldwide, royalty-free copyright license to
+reproduce, prepare derivative works of, publicly display, publicly perform,
+distribute and sublicense the Contribution of such Contributor, if any, and such
+derivative works, in source code and object code form.
+b) Subject to the terms of this Agreement, each Contributor hereby grants
+Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed
+Patents to make, use, sell, offer to sell, import and otherwise transfer the
+Contribution of such Contributor, if any, in source code and object code form.
+This patent license shall apply to the combination of the Contribution and the
+Program if, at the time the Contribution is added by the Contributor, such
+addition of the Contribution causes such combination to be covered by the
+Licensed Patents. The patent license shall not apply to any other combinations
+which include the Contribution. No hardware per se is licensed hereunder.
+c) Recipient understands that although each Contributor grants the licenses to
+its Contributions set forth herein, no assurances are provided by any
+Contributor that the Program does not infringe the patent or other intellectual
+property rights of any other entity. Each Contributor disclaims any liability to
+Recipient for claims brought by any other entity based on infringement of
+intellectual property rights or otherwise. As a condition to exercising the
+rights and licenses granted hereunder, each Recipient hereby assumes sole
+responsibility to secure any other intellectual property rights needed, if any.
+For example, if a third party patent license is required to allow Recipient to
+distribute the Program, it is Recipient's responsibility to acquire that license
+before distributing the Program.
+d) Each Contributor represents that to its knowledge it has sufficient copyright
+rights in its Contribution, if any, to grant the copyright license set forth in
+this Agreement.
+3. REQUIREMENTS
+
+A Contributor may choose to distribute the Program in object code form under its
+own license agreement, provided that:
+
+a) it complies with the terms and conditions of this Agreement; and
+b) its license agreement:
+i) effectively disclaims on behalf of all Contributors all warranties and
+conditions, express and implied, including warranties or conditions of title and
+non-infringement, and implied warranties or conditions of merchantability and
+fitness for a particular purpose;
+ii) effectively excludes on behalf of all Contributors all liability for
+damages, including direct, indirect, special, incidental and consequential
+damages, such as lost profits;
+iii) states that any provisions which differ from this Agreement are offered by
+that Contributor alone and not by any other party; and
+iv) states that source code for the Program is available from such Contributor,
+and informs licensees how to obtain it in a reasonable manner on or through a
+medium customarily used for software exchange.
+When the Program is made available in source code form:
+
+a) it must be made available under this Agreement; and
+b) a copy of this Agreement must be included with each copy of the Program.
+Contributors may not remove or alter any copyright notices contained within the
+Program.
+
+Each Contributor must identify itself as the originator of its Contribution, if
+any, in a manner that reasonably allows subsequent Recipients to identify the
+originator of the Contribution.
+
+4. COMMERCIAL DISTRIBUTION
+
+Commercial distributors of software may accept certain responsibilities with
+respect to end users, business partners and the like. While this license is
+intended to facilitate the commercial use of the Program, the Contributor who
+includes the Program in a commercial product offering should do so in a manner
+which does not create potential liability for other Contributors. Therefore, if
+a Contributor includes the Program in a commercial product offering, such
+Contributor ("Commercial Contributor") hereby agrees to defend and indemnify
+every other Contributor ("Indemnified Contributor") against any losses, damages
+and costs (collectively "Losses") arising from claims, lawsuits and other legal
+actions brought by a third party against the Indemnified Contributor to the
+extent caused by the acts or omissions of such Commercial Contributor in
+connection with its distribution of the Program in a commercial product
+offering. The obligations in this section do not apply to any claims or Losses
+relating to any actual or alleged intellectual property infringement. In order
+to qualify, an Indemnified Contributor must: a) promptly notify the Commercial
+Contributor in writing of such claim, and b) allow the Commercial Contributor to
+control, and cooperate with the Commercial Contributor in, the defense and any
+related settlement negotiations. The Indemnified Contributor may participate in
+any such claim at its own expense.
+
+For example, a Contributor might include the Program in a commercial product
+offering, Product X. That Contributor is then a Commercial Contributor. If that
+Commercial Contributor then makes performance claims, or offers warranties
+related to Product X, those performance claims and warranties are such
+Commercial Contributor's responsibility alone. Under this section, the
+Commercial Contributor would have to defend claims against the other
+Contributors related to those performance claims and warranties, and if a court
+requires any other Contributor to pay any damages as a result, the Commercial
+Contributor must pay those damages.
+
+5. NO WARRANTY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR
+IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE,
+NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each
+Recipient is solely responsible for determining the appropriateness of using and
+distributing the Program and assumes all risks associated with its exercise of
+rights under this Agreement , including but not limited to the risks and costs
+of program errors, compliance with applicable laws, damage to or loss of data,
+programs or equipment, and unavailability or interruption of operations.
+
+6. DISCLAIMER OF LIABILITY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY
+CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST
+PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY
+OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS
+GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+7. GENERAL
+
+If any provision of this Agreement is invalid or unenforceable under applicable
+law, it shall not affect the validity or enforceability of the remainder of the
+terms of this Agreement, and without further action by the parties hereto, such
+provision shall be reformed to the minimum extent necessary to make such
+provision valid and enforceable.
+
+If Recipient institutes patent litigation against any entity (including a
+cross-claim or counterclaim in a lawsuit) alleging that the Program itself
+(excluding combinations of the Program with other software or hardware)
+infringes such Recipient's patent(s), then such Recipient's rights granted under
+Section 2(b) shall terminate as of the date such litigation is filed.
+
+All Recipient's rights under this Agreement shall terminate if it fails to
+comply with any of the material terms or conditions of this Agreement and does
+not cure such failure in a reasonable period of time after becoming aware of
+such noncompliance. If all Recipient's rights under this Agreement terminate,
+Recipient agrees to cease use and distribution of the Program as soon as
+reasonably practicable. However, Recipient's obligations under this Agreement
+and any licenses granted by Recipient relating to the Program shall continue and
+survive.
+
+Everyone is permitted to copy and distribute copies of this Agreement, but in
+order to avoid inconsistency the Agreement is copyrighted and may only be
+modified in the following manner. The Agreement Steward reserves the right to
+publish new versions (including revisions) of this Agreement from time to time.
+No one other than the Agreement Steward has the right to modify this Agreement.
+The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation
+may assign the responsibility to serve as the Agreement Steward to a suitable
+separate entity. Each new version of the Agreement will be given a
+distinguishing version number. The Program (including Contributions) may always
+be distributed subject to the version of the Agreement under which it was
+received. In addition, after a new version of the Agreement is published,
+Contributor may elect to distribute the Program (including its Contributions)
+under the new version. Except as expressly stated in Sections 2(a) and 2(b)
+above, Recipient receives no rights or licenses to the intellectual property of
+any Contributor under this Agreement, whether expressly, by implication,
+estoppel or otherwise. All rights in the Program not expressly granted under
+this Agreement are reserved.
+
+This Agreement is governed by the laws of the State of New York and the
+intellectual property laws of the United States of America. No party to this
+Agreement will bring a legal action under this Agreement more than one year
+after the cause of action arose. Each party waives its rights to a jury trial in
+any resulting litigation.
+
+The binary distribution of this product bundles these dependencies under the
+following license:
+ASM Core 3.2
+JSch 0.1.51
+ParaNamer Core 2.3
+JLine 0.9.94
+leveldbjni-all 1.8
+Hamcrest Core 1.3
+xmlenc Library 0.52
+--------------------------------------------------------------------------------
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+    * Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above copyright
+      notice, this list of conditions and the following disclaimer in the
+      documentation and/or other materials provided with the distribution.
+    * Neither the name of the <organization> nor the
+      names of its contributors may be used to endorse or promote products
+      derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, T

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[02/51] [abbrv] hadoop git commit: YARN-4837. User facing aspects of 'AM blacklisting' feature need fixing. (vinodkv via wangda)

Posted by vv...@apache.org.
YARN-4837. User facing aspects of 'AM blacklisting' feature need fixing. (vinodkv via wangda)


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

Branch: refs/heads/YARN-3926
Commit: 620325e81696fca140195b74929ed9eda2d5eb16
Parents: be34e85
Author: Wangda Tan <wa...@apache.org>
Authored: Tue Jun 7 15:06:42 2016 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Tue Jun 7 15:06:42 2016 -0700

----------------------------------------------------------------------
 .../yarn/api/records/AMBlackListingRequest.java |  67 -----
 .../records/ApplicationSubmissionContext.java   |  23 --
 .../hadoop/yarn/conf/YarnConfiguration.java     |  25 +-
 .../src/main/proto/yarn_protos.proto            |   5 -
 .../yarn/conf/TestYarnConfigurationFields.java  |   7 +
 .../impl/pb/AMBlackListingRequestPBImpl.java    | 104 --------
 .../pb/ApplicationSubmissionContextPBImpl.java  |  40 ---
 .../src/main/resources/yarn-default.xml         |  19 --
 .../hadoop/yarn/api/TestPBImplRecords.java      |  10 -
 .../blacklist/BlacklistManager.java             |   9 +-
 .../blacklist/BlacklistUpdates.java             |  47 ----
 .../blacklist/DisabledBlacklistManager.java     |  12 +-
 .../blacklist/SimpleBlacklistManager.java       |  17 +-
 .../server/resourcemanager/rmapp/RMAppImpl.java |  79 ++----
 .../rmapp/attempt/RMAppAttempt.java             |   2 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         |  85 +++++--
 .../scheduler/AbstractYarnScheduler.java        |   2 +-
 .../scheduler/AppSchedulingInfo.java            |  74 +++---
 .../scheduler/SchedulerAppUtils.java            |  16 +-
 .../scheduler/SchedulerApplicationAttempt.java  |  33 ++-
 .../scheduler/capacity/CapacityScheduler.java   |  11 +-
 .../allocator/RegularContainerAllocator.java    |   2 +-
 .../scheduler/fair/FSLeafQueue.java             |   2 +-
 .../scheduler/fair/FairScheduler.java           |   8 +-
 .../scheduler/fifo/FifoScheduler.java           |  12 +-
 .../webapp/RMAppAttemptBlock.java               |   9 +-
 .../resourcemanager/webapp/RMAppBlock.java      |  13 +-
 .../resourcemanager/webapp/RMWebServices.java   |  21 +-
 .../webapp/dao/AMBlackListingRequestInfo.java   |  61 -----
 .../webapp/dao/AppAttemptInfo.java              |   8 +-
 .../dao/ApplicationSubmissionContextInfo.java   |  13 -
 .../TestNodeBlacklistingOnAMFailures.java       | 251 +++++++++++++++++++
 .../applicationsmanager/TestAMRestart.java      | 177 +------------
 .../blacklist/TestBlacklistManager.java         |  29 +--
 .../rmapp/TestRMAppTransitions.java             |  58 -----
 .../scheduler/TestAppSchedulingInfo.java        |  12 +-
 .../capacity/TestCapacityScheduler.java         |   8 +-
 .../scheduler/fair/TestFSAppAttempt.java        |  12 +-
 .../scheduler/fair/TestFairScheduler.java       |   9 +-
 .../TestRMWebServicesAppsModification.java      |  39 ++-
 40 files changed, 536 insertions(+), 895 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/AMBlackListingRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/AMBlackListingRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/AMBlackListingRequest.java
deleted file mode 100644
index 4aec2ba..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/AMBlackListingRequest.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.api.records;
-
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.util.Records;
-
-/**
- * Specific AMBlacklistingRequest from AM to enable/disable blacklisting.
- */
-@Public
-@Evolving
-public abstract class AMBlackListingRequest {
-
-  @Private
-  @Unstable
-  public static AMBlackListingRequest newInstance(
-      boolean isAMBlackListingEnabled, float disableFailureThreshold) {
-    AMBlackListingRequest blackListRequest = Records
-        .newRecord(AMBlackListingRequest.class);
-    blackListRequest.setBlackListingEnabled(isAMBlackListingEnabled);
-    blackListRequest
-        .setBlackListingDisableFailureThreshold(disableFailureThreshold);
-    return blackListRequest;
-  }
-
-  /**
-   * @return AM Blacklisting is enabled.
-   */
-  @Public
-  @Evolving
-  public abstract boolean isAMBlackListingEnabled();
-
-  /**
-   * @return AM Blacklisting disable failure threshold
-   */
-  @Public
-  @Evolving
-  public abstract float getBlackListingDisableFailureThreshold();
-
-  @Private
-  @Unstable
-  public abstract void setBlackListingEnabled(boolean isAMBlackListingEnabled);
-
-  @Private
-  @Unstable
-  public abstract void setBlackListingDisableFailureThreshold(
-      float disableFailureThreshold);
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
index fe833f7..21cd1bb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
@@ -535,27 +535,4 @@ public abstract class ApplicationSubmissionContext {
   @Public
   @Unstable
   public abstract void setReservationID(ReservationId reservationID);
-
-  /**
-   * Get AM Blacklisting request object to know whether application needs any
-   * specific blacklisting for AM Nodes.
-   *
-   * @return AMBlackListingRequest object which has blacklisting information.
-   */
-  @Public
-  @Unstable
-  public abstract AMBlackListingRequest getAMBlackListRequest();
-
-  /**
-   * Get AM Blacklisting request object to know whether application needs any
-   * specific blacklisting for AM Nodes.
-   *
-   * @param blackListRequest
-   *          object which has blacklisting information such as
-   *          "enable/disable AM blacklisting" and "disable failure threshold".
-   */
-  @Public
-  @Unstable
-  public abstract void setAMBlackListRequest(
-      AMBlackListingRequest blackListRequest);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/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 84ec894..0beec62 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
@@ -2553,14 +2553,25 @@ public class YarnConfiguration extends Configuration {
   public static final long DEFAULT_RM_NODE_LABELS_PROVIDER_FETCH_INTERVAL_MS =
       30 * 60 * 1000;
 
-  public static final String AM_BLACKLISTING_ENABLED =
-      YARN_PREFIX + "am.blacklisting.enabled";
-  public static final boolean DEFAULT_AM_BLACKLISTING_ENABLED = true;
-
-  public static final String AM_BLACKLISTING_DISABLE_THRESHOLD =
-      YARN_PREFIX + "am.blacklisting.disable-failure-threshold";
-  public static final float DEFAULT_AM_BLACKLISTING_DISABLE_THRESHOLD = 0.8f;
+  @Private
+  /**
+   * This is a private feature that isn't supposed to be used by end-users.
+   */
+  public static final String AM_SCHEDULING_NODE_BLACKLISTING_ENABLED =
+      RM_PREFIX + "am-scheduling.node-blacklisting-enabled";
+  @Private
+  public static final boolean DEFAULT_AM_SCHEDULING_NODE_BLACKLISTING_ENABLED =
+      true;
 
+  @Private
+  /**
+   * This is a private feature that isn't supposed to be used by end-users.
+   */
+  public static final String AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD =
+      RM_PREFIX + "am-scheduling.node-blacklisting-disable-threshold";
+  @Private
+  public static final float
+      DEFAULT_AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD = 0.8f;
 
   private static final String NM_SCRIPT_BASED_NODE_LABELS_PROVIDER_PREFIX =
       NM_NODE_LABELS_PROVIDER_PREFIX + "script.";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index ca33b28..0649f8e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -291,10 +291,6 @@ enum ExecutionTypeProto {
   OPPORTUNISTIC = 2;
 }
 
-message AMBlackListingRequestProto {
-  optional bool blacklisting_enabled = 1 [default = false];
-  optional float blacklisting_failure_threshold = 2;
-}
 ////////////////////////////////////////////////////////////////////////
 ////// From AM_RM_Protocol /////////////////////////////////////////////
 ////////////////////////////////////////////////////////////////////////
@@ -366,7 +362,6 @@ message ApplicationSubmissionContextProto {
   optional ReservationIdProto reservation_id = 15;
   optional string node_label_expression = 16;
   optional ResourceRequestProto am_container_resource_request = 17;
-  optional AMBlackListingRequestProto am_blacklisting_request = 18;
 }
 
 message LogAggregationContextProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index 61b698d..2c45b87 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -92,6 +92,13 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
         .add(YarnConfiguration.DEFAULT_AMRM_PROXY_INTERCEPTOR_CLASS_PIPELINE);
     configurationPropsToSkipCompare.add(YarnConfiguration.CURATOR_LEADER_ELECTOR);
 
+    // Ignore blacklisting nodes for AM failures feature since it is still a
+    // "work in progress"
+    configurationPropsToSkipCompare.add(YarnConfiguration.
+        AM_SCHEDULING_NODE_BLACKLISTING_ENABLED);
+    configurationPropsToSkipCompare.add(YarnConfiguration.
+        AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD);
+
     // Ignore all YARN Application Timeline Service (version 1) properties
     configurationPrefixToSkipCompare.add("yarn.timeline-service.");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/AMBlackListingRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/AMBlackListingRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/AMBlackListingRequestPBImpl.java
deleted file mode 100644
index 1d04dd2..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/AMBlackListingRequestPBImpl.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.api.records.impl.pb;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.records.AMBlackListingRequest;
-import org.apache.hadoop.yarn.proto.YarnProtos.AMBlackListingRequestProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.AMBlackListingRequestProtoOrBuilder;
-
-import com.google.protobuf.TextFormat;
-
-@Private
-@Unstable
-public class AMBlackListingRequestPBImpl extends AMBlackListingRequest {
-  AMBlackListingRequestProto proto = AMBlackListingRequestProto
-      .getDefaultInstance();
-  AMBlackListingRequestProto.Builder builder = null;
-  boolean viaProto = false;
-
-  public AMBlackListingRequestPBImpl() {
-    builder = AMBlackListingRequestProto.newBuilder();
-  }
-
-  public AMBlackListingRequestPBImpl(AMBlackListingRequestProto proto) {
-    this.proto = proto;
-    viaProto = true;
-  }
-
-  public AMBlackListingRequestProto getProto() {
-    proto = viaProto ? proto : builder.build();
-    viaProto = true;
-    return proto;
-  }
-
-  private void maybeInitBuilder() {
-    if (viaProto || builder == null) {
-      builder = AMBlackListingRequestProto.newBuilder(proto);
-    }
-    viaProto = false;
-  }
-
-  @Override
-  public boolean isAMBlackListingEnabled() {
-    AMBlackListingRequestProtoOrBuilder p = viaProto ? proto : builder;
-    return p.getBlacklistingEnabled();
-  }
-
-  @Override
-  public float getBlackListingDisableFailureThreshold() {
-    AMBlackListingRequestProtoOrBuilder p = viaProto ? proto : builder;
-    return p.getBlacklistingFailureThreshold();
-  }
-
-  @Override
-  public void setBlackListingEnabled(boolean isAMBlackListingEnabled) {
-    maybeInitBuilder();
-    builder.setBlacklistingEnabled(isAMBlackListingEnabled);
-  }
-
-  @Override
-  public void setBlackListingDisableFailureThreshold(
-      float disableFailureThreshold) {
-    maybeInitBuilder();
-    builder.setBlacklistingFailureThreshold(disableFailureThreshold);
-  }
-
-  @Override
-  public int hashCode() {
-    return getProto().hashCode();
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (other == null) {
-      return false;
-    }
-    if (other.getClass().isAssignableFrom(this.getClass())) {
-      return this.getProto().equals(this.getClass().cast(other).getProto());
-    }
-    return false;
-  }
-
-  @Override
-  public String toString() {
-    return TextFormat.shortDebugString(getProto());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
index b39258e..67e3a84 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
@@ -24,7 +24,6 @@ import java.util.Set;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.api.records.AMBlackListingRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
@@ -34,7 +33,6 @@ 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.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.proto.YarnProtos.AMBlackListingRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProtoOrBuilder;
@@ -65,7 +63,6 @@ extends ApplicationSubmissionContext {
   private ResourceRequest amResourceRequest = null;
   private LogAggregationContext logAggregationContext = null;
   private ReservationId reservationId = null;
-  private AMBlackListingRequest amBlackListRequest = null;
 
   public ApplicationSubmissionContextPBImpl() {
     builder = ApplicationSubmissionContextProto.newBuilder();
@@ -134,10 +131,6 @@ extends ApplicationSubmissionContext {
     if (this.reservationId != null) {
       builder.setReservationId(convertToProtoFormat(this.reservationId));
     }
-    if (this.amBlackListRequest != null) {
-      builder.setAmBlacklistingRequest(
-          convertToProtoFormat(this.amBlackListRequest));
-    }
   }
 
   private void mergeLocalToProto() {
@@ -420,29 +413,6 @@ extends ApplicationSubmissionContext {
     return p.getKeepContainersAcrossApplicationAttempts();
   }
 
-  @Override
-  public AMBlackListingRequest getAMBlackListRequest() {
-    ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
-    if (amBlackListRequest != null) {
-      return amBlackListRequest;
-    }
-    if (!p.hasAmBlacklistingRequest()) {
-      return null;
-    }
-    amBlackListRequest = convertFromProtoFormat(p.getAmBlacklistingRequest());
-    return amBlackListRequest;
-  }
-
-  @Override
-  public void setAMBlackListRequest(AMBlackListingRequest amBlackListRequest) {
-    maybeInitBuilder();
-    if (amBlackListRequest == null) {
-      builder.clearAmBlacklistingRequest();
-      return;
-    }
-    this.amBlackListRequest = amBlackListRequest;
-  }
-
   private PriorityPBImpl convertFromProtoFormat(PriorityProto p) {
     return new PriorityPBImpl(p);
   }
@@ -485,16 +455,6 @@ extends ApplicationSubmissionContext {
     return ((ResourcePBImpl)t).getProto();
   }
 
-  private AMBlackListingRequestPBImpl convertFromProtoFormat(
-      AMBlackListingRequestProto a) {
-    return new AMBlackListingRequestPBImpl(a);
-  }
-
-  private AMBlackListingRequestProto convertToProtoFormat(
-      AMBlackListingRequest a) {
-    return ((AMBlackListingRequestPBImpl) a).getProto();
-  }
-
   @Override
   public String getNodeLabelExpression() {
     ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/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 b3b2e2d..eabb679 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
@@ -2670,25 +2670,6 @@
     <value>4096</value>
   </property>
 
- <property>
-    <description>
-    Enable/disable blacklisting of hosts for AM based on AM failures on those
-    hosts.
-    </description>
-    <name>yarn.am.blacklisting.enabled</name>
-    <value>true</value>
-  </property>
-
-  <property>
-    <description>
-    Threshold of ratio number of NodeManager hosts that are allowed to be
-    blacklisted for AM. Beyond this ratio there is no blacklisting to avoid
-    danger of blacklisting the entire cluster.
-    </description>
-    <name>yarn.am.blacklisting.disable-failure-threshold</name>
-    <value>0.8f</value>
-  </property>
-
   <property>
     <description>
     Choose different implementation of node label's storage

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index 91d65b1..55b1233 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -109,7 +109,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersReso
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
-import org.apache.hadoop.yarn.api.records.AMBlackListingRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -155,7 +154,6 @@ import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
-import org.apache.hadoop.yarn.api.records.impl.pb.AMBlackListingRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptReportPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
@@ -190,7 +188,6 @@ import org.apache.hadoop.yarn.api.records.impl.pb.StrictPreemptionContractPBImpl
 import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.URLPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.YarnClusterMetricsPBImpl;
-import org.apache.hadoop.yarn.proto.YarnProtos.AMBlackListingRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptReportProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
@@ -515,7 +512,6 @@ public class TestPBImplRecords {
     generateByNewInstance(ResourceAllocationRequest.class);
     generateByNewInstance(ReservationAllocationState.class);
     generateByNewInstance(ResourceUtilization.class);
-    generateByNewInstance(AMBlackListingRequest.class);
   }
 
   private class GetSetPair {
@@ -1355,10 +1351,4 @@ public class TestPBImplRecords {
     validatePBImplRecord(CheckForDecommissioningNodesResponsePBImpl.class,
         CheckForDecommissioningNodesResponseProto.class);
   }
-
-  @Test
-  public void testAMBlackListingRequestPBImpl() throws Exception {
-    validatePBImplRecord(AMBlackListingRequestPBImpl.class,
-        AMBlackListingRequestProto.class);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/BlacklistManager.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/blacklist/BlacklistManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/BlacklistManager.java
index f03b421..f343603 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/BlacklistManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/BlacklistManager.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager.blacklist;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 
 /**
  * Tracks blacklists based on failures reported on nodes.
@@ -33,14 +34,14 @@ public interface BlacklistManager {
   void addNode(String node);
 
   /**
-   * Get {@link BlacklistUpdates} that indicate which nodes should be
+   * Get {@link ResourceBlacklistRequest} that indicate which nodes should be
    * added or to removed from the blacklist.
-   * @return {@link BlacklistUpdates}
+   * @return {@link ResourceBlacklistRequest}
    */
-  BlacklistUpdates getBlacklistUpdates();
+  ResourceBlacklistRequest getBlacklistUpdates();
 
   /**
-   * Refresh the number of nodemanager hosts available for scheduling.
+   * Refresh the number of NodeManagers available for scheduling.
    * @param nodeHostCount is the number of node hosts.
    */
   void refreshNodeHostCount(int nodeHostCount);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/BlacklistUpdates.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/blacklist/BlacklistUpdates.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/BlacklistUpdates.java
deleted file mode 100644
index c76dfb4..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/BlacklistUpdates.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.resourcemanager.blacklist;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-
-import java.util.List;
-
-/**
- * Class to track blacklist additions and removals.
- */
-@Private
-public class BlacklistUpdates {
-
-  private List<String> additions;
-  private List<String> removals;
-
-  public BlacklistUpdates(List<String> additions,
-      List<String> removals) {
-    this.additions = additions;
-    this.removals = removals;
-  }
-
-  public List<String> getAdditions() {
-    return additions;
-  }
-
-  public List<String> getRemovals() {
-    return removals;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/DisabledBlacklistManager.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/blacklist/DisabledBlacklistManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/DisabledBlacklistManager.java
index f155b45..8bb308d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/DisabledBlacklistManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/DisabledBlacklistManager.java
@@ -20,21 +20,23 @@ package org.apache.hadoop.yarn.server.resourcemanager.blacklist;
 
 import java.util.ArrayList;
 
+import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
+
 /**
  * A {@link BlacklistManager} that returns no blacklists.
  */
-public class DisabledBlacklistManager implements BlacklistManager{
+public class DisabledBlacklistManager implements BlacklistManager {
 
   private static final ArrayList<String> EMPTY_LIST = new ArrayList<String>();
-  private BlacklistUpdates noBlacklist =
-      new BlacklistUpdates(EMPTY_LIST, EMPTY_LIST);
+  private ResourceBlacklistRequest noBlacklist =
+      ResourceBlacklistRequest.newInstance(EMPTY_LIST, EMPTY_LIST);
 
   @Override
   public void addNode(String node) {
   }
 
   @Override
-  public BlacklistUpdates getBlacklistUpdates() {
+  public ResourceBlacklistRequest getBlacklistUpdates() {
     return noBlacklist;
   }
 
@@ -42,4 +44,4 @@ public class DisabledBlacklistManager implements BlacklistManager{
   public void refreshNodeHostCount(int nodeHostCount) {
     // Do nothing
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/SimpleBlacklistManager.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/blacklist/SimpleBlacklistManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/SimpleBlacklistManager.java
index c7bd0f8..f10e885 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/SimpleBlacklistManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/blacklist/SimpleBlacklistManager.java
@@ -18,14 +18,15 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.blacklist;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
+
 /**
  * Maintains a list of failed nodes and returns that as long as number of
  * blacklisted nodes is below a threshold percentage of total nodes. If more
@@ -58,8 +59,8 @@ public class SimpleBlacklistManager implements BlacklistManager {
   }
 
   @Override
-  public BlacklistUpdates getBlacklistUpdates() {
-    BlacklistUpdates ret;
+  public ResourceBlacklistRequest getBlacklistUpdates() {
+    ResourceBlacklistRequest ret;
     List<String> blacklist = new ArrayList<>(blacklistNodes);
     final int currentBlacklistSize = blacklist.size();
     final double failureThreshold = this.blacklistDisableFailureThreshold *
@@ -70,13 +71,15 @@ public class SimpleBlacklistManager implements BlacklistManager {
             "failure threshold ratio " + blacklistDisableFailureThreshold +
             " out of total usable nodes " + numberOfNodeManagerHosts);
       }
-      ret = new BlacklistUpdates(blacklist, EMPTY_LIST);
+      ret = ResourceBlacklistRequest.newInstance(blacklist, EMPTY_LIST);
     } else {
       LOG.warn("Ignoring Blacklists, blacklist size " + currentBlacklistSize
           + " is more than failure threshold ratio "
           + blacklistDisableFailureThreshold + " out of total usable nodes "
           + numberOfNodeManagerHosts);
-      ret = new BlacklistUpdates(EMPTY_LIST, blacklist);
+      // TODO: After the threshold hits, we will keep sending a long list
+      // every time a new AM is to be scheduled.
+      ret = ResourceBlacklistRequest.newInstance(EMPTY_LIST, blacklist);
     }
     return ret;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/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 07d5a74..53d8ae5 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
@@ -391,8 +391,8 @@ public class RMAppImpl implements RMApp, Recoverable {
                                                                  stateMachine;
 
   private static final int DUMMY_APPLICATION_ATTEMPT_NUMBER = -1;
-  private static final float MINIMUM_THRESHOLD_VALUE = 0.0f;
-  private static final float MAXIMUM_THRESHOLD_VALUE = 1.0f;
+  private static final float MINIMUM_AM_BLACKLIST_THRESHOLD_VALUE = 0.0f;
+  private static final float MAXIMUM_AM_BLACKLIST_THRESHOLD_VALUE = 1.0f;
 
   public RMAppImpl(ApplicationId applicationId, RMContext rmContext,
       Configuration config, String name, String user, String queue,
@@ -471,42 +471,24 @@ public class RMAppImpl implements RMApp, Recoverable {
         YarnConfiguration.RM_MAX_LOG_AGGREGATION_DIAGNOSTICS_IN_MEMORY,
         YarnConfiguration.DEFAULT_RM_MAX_LOG_AGGREGATION_DIAGNOSTICS_IN_MEMORY);
 
-    // amBlacklistingEnabled can be configured globally and by each
-    // application.
-    // Case 1: If AMBlackListRequest is available in submission context, we
-    // will consider only app level request (RM level configuration will be
-    // skipped).
-    // Case 2: AMBlackListRequest is available in submission context and
-    // amBlacklisting is disabled. In this case, AM blacklisting wont be
-    // enabled for this app even if this feature is enabled in RM level.
-    // Case 3: AMBlackListRequest is not available through submission context.
-    // RM level AM black listing configuration will be considered.
-    if (null != submissionContext.getAMBlackListRequest()) {
-      amBlacklistingEnabled = submissionContext.getAMBlackListRequest()
-          .isAMBlackListingEnabled();
-      blacklistDisableThreshold = 0.0f;
-      if (amBlacklistingEnabled) {
-        blacklistDisableThreshold = submissionContext.getAMBlackListRequest()
-            .getBlackListingDisableFailureThreshold();
-
-        // Verify whether blacklistDisableThreshold is valid. And for invalid
-        // threshold, reset to global level blacklistDisableThreshold
-        // configured.
-        if (blacklistDisableThreshold < MINIMUM_THRESHOLD_VALUE
-            || blacklistDisableThreshold > MAXIMUM_THRESHOLD_VALUE) {
-          blacklistDisableThreshold = conf.getFloat(
-              YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD,
-              YarnConfiguration.DEFAULT_AM_BLACKLISTING_DISABLE_THRESHOLD);
-        }
-      }
-    } else {
-      amBlacklistingEnabled = conf.getBoolean(
-          YarnConfiguration.AM_BLACKLISTING_ENABLED,
-          YarnConfiguration.DEFAULT_AM_BLACKLISTING_ENABLED);
-      if (amBlacklistingEnabled) {
-        blacklistDisableThreshold = conf.getFloat(
-            YarnConfiguration.AM_BLACKLISTING_DISABLE_THRESHOLD,
-            YarnConfiguration.DEFAULT_AM_BLACKLISTING_DISABLE_THRESHOLD);
+    // amBlacklistingEnabled can be configured globally
+    // Just use the global values
+    amBlacklistingEnabled =
+        conf.getBoolean(
+          YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_ENABLED,
+          YarnConfiguration.DEFAULT_AM_SCHEDULING_NODE_BLACKLISTING_ENABLED);
+    if (amBlacklistingEnabled) {
+      blacklistDisableThreshold = conf.getFloat(
+          YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD,
+          YarnConfiguration.
+          DEFAULT_AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD);
+      // Verify whether blacklistDisableThreshold is valid. And for invalid
+      // threshold, reset to global level blacklistDisableThreshold
+      // configured.
+      if (blacklistDisableThreshold < MINIMUM_AM_BLACKLIST_THRESHOLD_VALUE ||
+          blacklistDisableThreshold > MAXIMUM_AM_BLACKLIST_THRESHOLD_VALUE) {
+        blacklistDisableThreshold = YarnConfiguration.
+            DEFAULT_AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD;
       }
     }
   }
@@ -877,15 +859,16 @@ public class RMAppImpl implements RMApp, Recoverable {
   }
 
   private void createNewAttempt(ApplicationAttemptId appAttemptId) {
-    BlacklistManager currentAMBlacklist;
+    BlacklistManager currentAMBlacklistManager;
     if (currentAttempt != null) {
-      currentAMBlacklist = currentAttempt.getAMBlacklist();
+      // Transfer over the blacklist from the previous app-attempt.
+      currentAMBlacklistManager = currentAttempt.getAMBlacklistManager();
     } else {
       if (amBlacklistingEnabled) {
-        currentAMBlacklist = new SimpleBlacklistManager(
+        currentAMBlacklistManager = new SimpleBlacklistManager(
             scheduler.getNumClusterNodes(), blacklistDisableThreshold);
       } else {
-        currentAMBlacklist = new DisabledBlacklistManager();
+        currentAMBlacklistManager = new DisabledBlacklistManager();
       }
     }
     RMAppAttempt attempt =
@@ -896,7 +879,7 @@ public class RMAppImpl implements RMApp, Recoverable {
           // hardware error and NM resync) + 1) equal to the max-attempt
           // limit.
           maxAppAttempts == (getNumFailedAppAttempts() + 1), amReq,
-          currentAMBlacklist);
+          currentAMBlacklistManager);
     attempts.put(appAttemptId, attempt);
     currentAttempt = attempt;
   }
@@ -1825,16 +1808,6 @@ public class RMAppImpl implements RMApp, Recoverable {
     rmContext.getSystemMetricsPublisher().appCreated(app, startTime);
   }
 
-  @VisibleForTesting
-  public boolean isAmBlacklistingEnabled() {
-    return amBlacklistingEnabled;
-  }
-
-  @VisibleForTesting
-  public float getAmBlacklistingDisableThreshold() {
-    return blacklistDisableThreshold;
-  }
-
   @Private
   @VisibleForTesting
   public int getNextAttemptId() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.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/attempt/RMAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java
index f2e60d1..cfd91e9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java
@@ -190,7 +190,7 @@ public interface RMAppAttempt extends EventHandler<RMAppAttemptEvent> {
    * Get the {@link BlacklistManager} that manages blacklists for AM failures
    * @return the {@link BlacklistManager} that tracks AM failures.
    */
-  BlacklistManager getAMBlacklist();
+  BlacklistManager getAMBlacklistManager();
 
   /**
    * the start time of the application.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.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/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
index 75090fe..d210b53 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -73,7 +74,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.blacklist.BlacklistManager;
-import org.apache.hadoop.yarn.server.resourcemanager.blacklist.BlacklistUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.blacklist.DisabledBlacklistManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
@@ -492,7 +492,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       ApplicationMasterService masterService,
       ApplicationSubmissionContext submissionContext,
       Configuration conf, boolean maybeLastAttempt, ResourceRequest amReq,
-      BlacklistManager amBlacklist) {
+      BlacklistManager amBlacklistManager) {
     this.conf = conf;
     this.applicationAttemptId = appAttemptId;
     this.rmContext = rmContext;
@@ -511,9 +511,9 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
 
     this.attemptMetrics =
         new RMAppAttemptMetrics(applicationAttemptId, rmContext);
-    
+
     this.amReq = amReq;
-    this.blacklistedNodesForAM = amBlacklist;
+    this.blacklistedNodesForAM = amBlacklistManager;
   }
 
   @Override
@@ -1031,15 +1031,15 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
         appAttempt.amReq.setResourceName(ResourceRequest.ANY);
         appAttempt.amReq.setRelaxLocality(true);
 
-        appAttempt.getAMBlacklist().refreshNodeHostCount(
+        appAttempt.getAMBlacklistManager().refreshNodeHostCount(
             appAttempt.scheduler.getNumClusterNodes());
 
-        BlacklistUpdates amBlacklist = appAttempt.getAMBlacklist()
-            .getBlacklistUpdates();
+        ResourceBlacklistRequest amBlacklist =
+            appAttempt.getAMBlacklistManager().getBlacklistUpdates();
         if (LOG.isDebugEnabled()) {
           LOG.debug("Using blacklist for AM: additions(" +
-              amBlacklist.getAdditions() + ") and removals(" +
-              amBlacklist.getRemovals() + ")");
+              amBlacklist.getBlacklistAdditions() + ") and removals(" +
+              amBlacklist.getBlacklistRemovals() + ")");
         }
         // AM resource has been checked when submission
         Allocation amContainerAllocation =
@@ -1047,8 +1047,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
                 appAttempt.applicationAttemptId,
                 Collections.singletonList(appAttempt.amReq),
                 EMPTY_CONTAINER_RELEASE_LIST,
-                amBlacklist.getAdditions(),
-                amBlacklist.getRemovals(), null, null);
+                amBlacklist.getBlacklistAdditions(),
+                amBlacklist.getBlacklistRemovals(), null, null);
         if (amContainerAllocation != null
             && amContainerAllocation.getContainers() != null) {
           assert (amContainerAllocation.getContainers().size() == 0);
@@ -1481,9 +1481,36 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     }
   }
 
-  private boolean shouldCountTowardsNodeBlacklisting(int exitStatus) {
-    return !(exitStatus == ContainerExitStatus.SUCCESS
-        || exitStatus == ContainerExitStatus.PREEMPTED);
+  private static boolean shouldCountTowardsNodeBlacklisting(int exitStatus) {
+    switch (exitStatus) {
+    case ContainerExitStatus.PREEMPTED:
+    case ContainerExitStatus.KILLED_BY_RESOURCEMANAGER:
+    case ContainerExitStatus.KILLED_BY_APPMASTER:
+    case ContainerExitStatus.KILLED_AFTER_APP_COMPLETION:
+    case ContainerExitStatus.ABORTED:
+      // Neither the app's fault nor the system's fault. This happens by design,
+      // so no need for skipping nodes
+      return false;
+    case ContainerExitStatus.DISKS_FAILED:
+      // This container is marked with this exit-status means that the node is
+      // already marked as unhealthy given that most of the disks failed. So, no
+      // need for any explicit skipping of nodes.
+      return false;
+    case ContainerExitStatus.KILLED_EXCEEDED_VMEM:
+    case ContainerExitStatus.KILLED_EXCEEDED_PMEM:
+      // No point in skipping the node as it's not the system's fault
+      return false;
+    case ContainerExitStatus.SUCCESS:
+      return false;
+    case ContainerExitStatus.INVALID:
+      // Ideally, this shouldn't be considered for skipping a node. But in
+      // reality, it seems like there are cases where we are not setting
+      // exit-code correctly and so it's better to be conservative. See
+      // YARN-4284.
+      return true;
+    default:
+      return true;
+    }
   }
 
   private static final class UnmanagedAMAttemptSavedTransition
@@ -1803,7 +1830,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       if (appAttempt.masterContainer != null
           && appAttempt.masterContainer.getId().equals(
               containerStatus.getContainerId())) {
-        appAttempt.sendAMContainerToNM(appAttempt, containerFinishedEvent);
+        appAttempt.amContainerFinished(appAttempt, containerFinishedEvent);
 
         // Remember the follow up transition and save the final attempt state.
         appAttempt.rememberTargetTransitionsAndStoreState(event,
@@ -1848,13 +1875,17 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
 
   // Add am container to the list so that am container instance will be
   // removed from NMContext.
-  private void sendAMContainerToNM(RMAppAttemptImpl appAttempt,
+  private static void amContainerFinished(RMAppAttemptImpl appAttempt,
       RMAppAttemptContainerFinishedEvent containerFinishedEvent) {
+
     NodeId nodeId = containerFinishedEvent.getNodeId();
-    if (containerFinishedEvent.getContainerStatus() != null) {
-      if (shouldCountTowardsNodeBlacklisting(containerFinishedEvent
-          .getContainerStatus().getExitStatus())) {
-        appAttempt.addAMNodeToBlackList(containerFinishedEvent.getNodeId());
+
+    ContainerStatus containerStatus =
+        containerFinishedEvent.getContainerStatus();
+    if (containerStatus != null) {
+      int exitStatus = containerStatus.getExitStatus();
+      if (shouldCountTowardsNodeBlacklisting(exitStatus)) {
+        appAttempt.addAMNodeToBlackList(nodeId);
       }
     } else {
       LOG.warn("No ContainerStatus in containerFinishedEvent");
@@ -1862,14 +1893,13 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
 
     if (!appAttempt.getSubmissionContext()
         .getKeepContainersAcrossApplicationAttempts()) {
-      finishedContainersSentToAM.putIfAbsent(nodeId,
+      appAttempt.finishedContainersSentToAM.putIfAbsent(nodeId,
           new ArrayList<ContainerStatus>());
-      appAttempt.finishedContainersSentToAM.get(nodeId).add(
-          containerFinishedEvent.getContainerStatus());
+      appAttempt.finishedContainersSentToAM.get(nodeId).add(containerStatus);
       appAttempt.sendFinishedContainersToNM();
     } else {
       appAttempt.sendFinishedAMContainerToNM(nodeId,
-          containerFinishedEvent.getContainerStatus().getContainerId());
+          containerStatus.getContainerId());
     }
   }
 
@@ -1884,7 +1914,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
   }
 
   @Override
-  public BlacklistManager getAMBlacklist() {
+  public BlacklistManager getAMBlacklistManager() {
     return blacklistedNodesForAM;
   }
 
@@ -1943,7 +1973,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
           containerStatus.getContainerId())) {
         new FinalTransition(RMAppAttemptState.FINISHED).transition(
             appAttempt, containerFinishedEvent);
-        appAttempt.sendAMContainerToNM(appAttempt, containerFinishedEvent);
+        appAttempt.amContainerFinished(appAttempt, containerFinishedEvent);
         return RMAppAttemptState.FINISHED;
       }
       // Add all finished containers so that they can be acked to NM.
@@ -1968,7 +1998,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       // Thus, we still return FINAL_SAVING state here.
       if (appAttempt.masterContainer.getId().equals(
         containerStatus.getContainerId())) {
-        appAttempt.sendAMContainerToNM(appAttempt, containerFinishedEvent);
+
+        appAttempt.amContainerFinished(appAttempt, containerFinishedEvent);
 
         if (appAttempt.targetedFinalState.equals(RMAppAttemptState.FAILED)
             || appAttempt.targetedFinalState.equals(RMAppAttemptState.KILLED)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index 354dcb2..3066339 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -178,7 +178,7 @@ public abstract class AbstractYarnScheduler
     NodeFilter nodeFilter = new NodeFilter() {
       @Override
       public boolean accept(SchedulerNode node) {
-        return SchedulerAppUtils.isBlacklisted(app, node, LOG);
+        return SchedulerAppUtils.isPlaceBlacklisted(app, node, LOG);
       }
     };
     return nodeTracker.getNodes(nodeFilter);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
index 463bebd..8d42c97 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
@@ -73,9 +73,13 @@ public class AppSchedulingInfo {
   private ActiveUsersManager activeUsersManager;
   private boolean pending = true; // whether accepted/allocated by scheduler
   private ResourceUsage appResourceUsage;
+
   private AtomicBoolean userBlacklistChanged = new AtomicBoolean(false);
-  private final Set<String> amBlacklist = new HashSet<>();
-  private Set<String> userBlacklist = new HashSet<>();
+  // Set of places (nodes / racks) blacklisted by the system. Today, this only
+  // has places blacklisted for AM containers.
+  private final Set<String> placesBlacklistedBySystem = new HashSet<>();
+  private Set<String> placesBlacklistedByApp = new HashSet<>();
+
   private Set<String> requestedPartitions = new HashSet<>();
 
   final Set<Priority> priorities = new TreeSet<>(COMPARATOR);
@@ -447,32 +451,38 @@ public class AppSchedulingInfo {
   }
 
   /**
-   * The ApplicationMaster is updating the userBlacklist used for containers
-   * other than AMs.
+   * The ApplicationMaster is updating the placesBlacklistedByApp used for
+   * containers other than AMs.
    *
-   * @param blacklistAdditions resources to be added to the userBlacklist
-   * @param blacklistRemovals resources to be removed from the userBlacklist
+   * @param blacklistAdditions
+   *          resources to be added to the userBlacklist
+   * @param blacklistRemovals
+   *          resources to be removed from the userBlacklist
    */
-  public void updateBlacklist(
+  public void updatePlacesBlacklistedByApp(
       List<String> blacklistAdditions, List<String> blacklistRemovals) {
-    if (updateUserOrAMBlacklist(userBlacklist, blacklistAdditions,
+    if (updateBlacklistedPlaces(placesBlacklistedByApp, blacklistAdditions,
         blacklistRemovals)) {
       userBlacklistChanged.set(true);
     }
   }
 
   /**
-   * RM is updating blacklist for AM containers.
-   * @param blacklistAdditions resources to be added to the amBlacklist
-   * @param blacklistRemovals resources to be added to the amBlacklist
+   * Update the list of places that are blacklisted by the system. Today the
+   * system only blacklists places when it sees that AMs failed there
+   *
+   * @param blacklistAdditions
+   *          resources to be added to placesBlacklistedBySystem
+   * @param blacklistRemovals
+   *          resources to be removed from placesBlacklistedBySystem
    */
-  public void updateAMBlacklist(
+  public void updatePlacesBlacklistedBySystem(
       List<String> blacklistAdditions, List<String> blacklistRemovals) {
-    updateUserOrAMBlacklist(amBlacklist, blacklistAdditions,
+    updateBlacklistedPlaces(placesBlacklistedBySystem, blacklistAdditions,
         blacklistRemovals);
   }
 
-  boolean updateUserOrAMBlacklist(Set<String> blacklist,
+  private static boolean updateBlacklistedPlaces(Set<String> blacklist,
       List<String> blacklistAdditions, List<String> blacklistRemovals) {
     boolean changed = false;
     synchronized (blacklist) {
@@ -481,9 +491,7 @@ public class AppSchedulingInfo {
       }
 
       if (blacklistRemovals != null) {
-        if (blacklist.removeAll(blacklistRemovals)) {
-          changed = true;
-        }
+        changed = blacklist.removeAll(blacklistRemovals) || changed;
       }
     }
     return changed;
@@ -522,20 +530,24 @@ public class AppSchedulingInfo {
   }
 
   /**
-   * Returns if the node is either blacklisted by the user or the system
-   * @param resourceName the resourcename
-   * @param useAMBlacklist true if it should check amBlacklist
+   * Returns if the place (node/rack today) is either blacklisted by the
+   * application (user) or the system
+   *
+   * @param resourceName
+   *          the resourcename
+   * @param blacklistedBySystem
+   *          true if it should check amBlacklist
    * @return true if its blacklisted
    */
-  public boolean isBlacklisted(String resourceName,
-      boolean useAMBlacklist) {
-    if (useAMBlacklist){
-      synchronized (amBlacklist) {
-        return amBlacklist.contains(resourceName);
+  public boolean isPlaceBlacklisted(String resourceName,
+      boolean blacklistedBySystem) {
+    if (blacklistedBySystem){
+      synchronized (placesBlacklistedBySystem) {
+        return placesBlacklistedBySystem.contains(resourceName);
       }
     } else {
-      synchronized (userBlacklist) {
-        return userBlacklist.contains(resourceName);
+      synchronized (placesBlacklistedByApp) {
+        return placesBlacklistedByApp.contains(resourceName);
       }
     }
   }
@@ -773,12 +785,12 @@ public class AppSchedulingInfo {
   }
 
   public Set<String> getBlackList() {
-    return this.userBlacklist;
+    return this.placesBlacklistedByApp;
   }
 
   public Set<String> getBlackListCopy() {
-    synchronized (userBlacklist) {
-      return new HashSet<>(this.userBlacklist);
+    synchronized (placesBlacklistedByApp) {
+      return new HashSet<>(this.placesBlacklistedByApp);
     }
   }
 
@@ -786,7 +798,7 @@ public class AppSchedulingInfo {
       AppSchedulingInfo appInfo) {
     // This should not require locking the userBlacklist since it will not be
     // used by this instance until after setCurrentAppAttempt.
-    this.userBlacklist = appInfo.getBlackList();
+    this.placesBlacklistedByApp = appInfo.getBlackList();
   }
 
   public synchronized void recoverContainer(RMContainer rmContainer) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerAppUtils.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/SchedulerAppUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerAppUtils.java
index 36a1244..631da67 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerAppUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerAppUtils.java
@@ -22,20 +22,20 @@ import org.apache.commons.logging.Log;
 
 public class SchedulerAppUtils {
 
-  public static  boolean isBlacklisted(SchedulerApplicationAttempt application,
-      SchedulerNode node, Log LOG) {
-    if (application.isBlacklisted(node.getNodeName())) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Skipping 'host' " + node.getNodeName() +
+  public static boolean isPlaceBlacklisted(
+      SchedulerApplicationAttempt application, SchedulerNode node, Log log) {
+    if (application.isPlaceBlacklisted(node.getNodeName())) {
+      if (log.isDebugEnabled()) {
+        log.debug("Skipping 'host' " + node.getNodeName() +
             " for " + application.getApplicationId() +
             " since it has been blacklisted");
       }
       return true;
     }
 
-    if (application.isBlacklisted(node.getRackName())) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Skipping 'rack' " + node.getRackName() +
+    if (application.isPlaceBlacklisted(node.getRackName())) {
+      if (log.isDebugEnabled()) {
+        log.debug("Skipping 'rack' " + node.getRackName() +
             " for " + application.getApplicationId() +
             " since it has been blacklisted");
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index ffb8657..b48b272 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -592,27 +592,26 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return (!unmanagedAM && appAttempt.getMasterContainer() == null);
   }
 
-  // Blacklist used for user containers
-  public synchronized void updateBlacklist(
-      List<String> blacklistAdditions, List<String> blacklistRemovals) {
+  public synchronized void updateBlacklist(List<String> blacklistAdditions,
+      List<String> blacklistRemovals) {
     if (!isStopped) {
-      this.appSchedulingInfo.updateBlacklist(
-          blacklistAdditions, blacklistRemovals);
-    }
-  }
-
-  // Blacklist used for AM containers
-  public synchronized void updateAMBlacklist(
-      List<String> blacklistAdditions, List<String> blacklistRemovals) {
-    if (!isStopped) {
-      this.appSchedulingInfo.updateAMBlacklist(
-          blacklistAdditions, blacklistRemovals);
+      if (isWaitingForAMContainer()) {
+        // The request is for the AM-container, and the AM-container is launched
+        // by the system. So, update the places that are blacklisted by system
+        // (as opposed to those blacklisted by the application).
+        this.appSchedulingInfo.updatePlacesBlacklistedBySystem(
+            blacklistAdditions, blacklistRemovals);
+      } else {
+        this.appSchedulingInfo.updatePlacesBlacklistedByApp(blacklistAdditions,
+            blacklistRemovals);
+      }
     }
   }
 
-  public boolean isBlacklisted(String resourceName) {
-    boolean useAMBlacklist = isWaitingForAMContainer();
-    return this.appSchedulingInfo.isBlacklisted(resourceName, useAMBlacklist);
+  public boolean isPlaceBlacklisted(String resourceName) {
+    boolean forAMContainer = isWaitingForAMContainer();
+    return this.appSchedulingInfo.isPlaceBlacklisted(resourceName,
+      forAMContainer);
   }
 
   public synchronized int addMissedNonPartitionedRequestSchedulingOpportunity(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.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/CapacityScheduler.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/CapacityScheduler.java
index 920e983..ee62a70 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/CapacityScheduler.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/CapacityScheduler.java
@@ -996,15 +996,8 @@ public class CapacityScheduler extends
           application.showRequests();
         }
       }
-      
-      if (application.isWaitingForAMContainer()) {
-        // Allocate is for AM and update AM blacklist for this
-        application.updateAMBlacklist(
-            blacklistAdditions, blacklistRemovals);
-      } else {
-        application.updateBlacklist(blacklistAdditions, blacklistRemovals);
-      }
-      
+
+      application.updateBlacklist(blacklistAdditions, blacklistRemovals);
 
       allocation = application.getAllocation(getResourceCalculator(),
           getClusterResource(), getMinimumResourceCapability());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.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/allocator/RegularContainerAllocator.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/allocator/RegularContainerAllocator.java
index b2d4bbe..aae5292 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/allocator/RegularContainerAllocator.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/allocator/RegularContainerAllocator.java
@@ -81,7 +81,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
   private ContainerAllocation preCheckForNewContainer(Resource clusterResource,
       FiCaSchedulerNode node, SchedulingMode schedulingMode,
       ResourceLimits resourceLimits, Priority priority) {
-    if (SchedulerAppUtils.isBlacklisted(application, node, LOG)) {
+    if (SchedulerAppUtils.isPlaceBlacklisted(application, node, LOG)) {
       application.updateAppSkipNodeDiagnostics(
           CSAMContainerLaunchDiagnosticsConstants.SKIP_AM_ALLOCATION_IN_BLACK_LISTED_NODE);
       return ContainerAllocation.APP_SKIPPED;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
index a398906..d8b51f7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
@@ -332,7 +332,7 @@ public class FSLeafQueue extends FSQueue {
       readLock.unlock();
     }
     for (FSAppAttempt sched : pendingForResourceApps) {
-      if (SchedulerAppUtils.isBlacklisted(sched, node, LOG)) {
+      if (SchedulerAppUtils.isPlaceBlacklisted(sched, node, LOG)) {
         continue;
       }
       assigned = sched.assignContainer(node);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index c8e8406..bc953ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -993,13 +993,7 @@ public class FairScheduler extends
         preemptionContainerIds.add(container.getContainerId());
       }
 
-      if (application.isWaitingForAMContainer()) {
-        // Allocate is for AM and update AM blacklist for this
-        application.updateAMBlacklist(
-            blacklistAdditions, blacklistRemovals);
-      } else {
-        application.updateBlacklist(blacklistAdditions, blacklistRemovals);
-      }
+      application.updateBlacklist(blacklistAdditions, blacklistRemovals);
 
       List<Container> newlyAllocatedContainers =
           application.pullNewlyAllocatedContainers();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
index 796b0cf..eaab495 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
@@ -375,13 +375,7 @@ public class FifoScheduler extends
             " #ask=" + ask.size());
       }
 
-      if (application.isWaitingForAMContainer()) {
-        // Allocate is for AM and update AM blacklist for this
-        application.updateAMBlacklist(
-            blacklistAdditions, blacklistRemovals);
-      } else {
-        application.updateBlacklist(blacklistAdditions, blacklistRemovals);
-      }
+      application.updateBlacklist(blacklistAdditions, blacklistRemovals);
 
       Resource headroom = application.getHeadroom();
       application.setApplicationHeadroomForMetrics(headroom);
@@ -516,10 +510,10 @@ public class FifoScheduler extends
       application.showRequests();
       synchronized (application) {
         // Check if this resource is on the blacklist
-        if (SchedulerAppUtils.isBlacklisted(application, node, LOG)) {
+        if (SchedulerAppUtils.isPlaceBlacklisted(application, node, LOG)) {
           continue;
         }
-        
+
         for (Priority priority : application.getPriorities()) {
           int maxContainers = 
             getMaxAllocatableContainers(application, priority, node, 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.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/RMAppAttemptBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.java
index 6fef367..d2da8e3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.java
@@ -220,8 +220,9 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
     String appBlacklistedNodes =
         getNodeString(rmAppAttempt.getBlacklistedNodes());
     // nodes which are blacklisted by the RM for AM launches
-    String rmBlackListedNodes = getNodeString(
-        rmAppAttempt.getAMBlacklist().getBlacklistUpdates().getAdditions());
+    String rmBlackListedNodes =
+        getNodeString(rmAppAttempt.getAMBlacklistManager()
+          .getBlacklistUpdates().getBlacklistAdditions());
 
     info("Application Attempt Overview")
       ._(
@@ -256,8 +257,8 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
         "Diagnostics Info:",
         appAttempt.getDiagnosticsInfo() == null ? "" : appAttempt
           .getDiagnosticsInfo())
-      ._("Application Blacklisted Nodes:", appBlacklistedNodes)
-      ._("RM Blacklisted Nodes(for AM launches)", rmBlackListedNodes);
+      ._("Nodes blacklisted by the application:", appBlacklistedNodes)
+      ._("Nodes blacklisted by the system:", rmBlackListedNodes);
   }
 
   private String getNodeString(Collection<String> nodes) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.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/RMAppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
index 2d822c1..cfa21dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
@@ -113,10 +113,10 @@ public class RMAppBlock extends AppBlock{
     Hamlet.TBODY<Hamlet.TABLE<Hamlet>> tbody =
         html.table("#attempts").thead().tr().th(".id", "Attempt ID")
             .th(".started", "Started").th(".node", "Node").th(".logs", "Logs")
-            .th(".appBlacklistednodes", "Nodes black listed by the application",
-                "App Blacklisted Nodes")
-            .th(".rmBlacklistednodes", "Nodes black listed by the RM for the"
-                + " app", "RM Blacklisted Nodes")._()._().tbody();
+            .th(".appBlacklistednodes", "Nodes blacklisted by the application",
+                "Nodes blacklisted by the app")
+            .th(".rmBlacklistednodes", "Nodes blacklisted by the RM for the"
+                + " app", "Nodes blacklisted by the system")._()._().tbody();
 
     RMApp rmApp = this.rm.getRMContext().getRMApps().get(this.appID);
     if (rmApp == null) {
@@ -136,8 +136,9 @@ public class RMAppBlock extends AppBlock{
       // nodes which are blacklisted by the application
       String appBlacklistedNodesCount = String.valueOf(nodes.size());
       // nodes which are blacklisted by the RM for AM launches
-      String rmBlacklistedNodesCount = String.valueOf(rmAppAttempt
-          .getAMBlacklist().getBlacklistUpdates().getAdditions().size());
+      String rmBlacklistedNodesCount =
+          String.valueOf(rmAppAttempt.getAMBlacklistManager()
+            .getBlacklistUpdates().getBlacklistAdditions().size());
       String nodeLink = attemptInfo.getNodeHttpAddress();
       if (nodeLink != null) {
         nodeLink = WebAppUtils.getHttpSchemePrefix(conf) + nodeLink;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.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/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 7de1d46..d05d952 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -85,12 +85,15 @@ import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationListRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityRequest;
-import org.apache.hadoop.yarn.api.records.AMBlackListingRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
@@ -130,7 +133,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AMBlackListingRequestInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
@@ -168,10 +170,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationReque
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionRequestInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateRequestInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateResponseInfo;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
@@ -1499,10 +1497,6 @@ public class RMWebServices extends WebServices {
           reservationIdStr);
       appContext.setReservationID(reservationId);
     }
-    if (newApp.getAMBlackListingRequestInfo() != null) {
-      appContext.setAMBlackListRequest(createAMBlackListingRequest(
-          newApp.getAMBlackListingRequestInfo()));
-    }
     return appContext;
   }
 
@@ -1651,13 +1645,6 @@ public class RMWebServices extends WebServices {
         logAggregationContextInfo.getLogAggregationPolicyParameters());
   }
 
-  private AMBlackListingRequest createAMBlackListingRequest(
-      AMBlackListingRequestInfo amBlackListingRequestInfo) {
-    return AMBlackListingRequest.newInstance(
-        amBlackListingRequestInfo.getAMBlackListingEnabled(),
-        amBlackListingRequestInfo.getBlackListingDisableFailureThreshold());
-  }
-
   @POST
   @Path("/delegation-token")
   @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })

http://git-wip-us.apache.org/repos/asf/hadoop/blob/620325e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AMBlackListingRequestInfo.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/AMBlackListingRequestInfo.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/AMBlackListingRequestInfo.java
deleted file mode 100644
index 0084b0b..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AMBlackListingRequestInfo.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
-
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlElement;
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- * Simple class to allow users to send information required to create a
- * AMBlackListingRequest which can then be used as part of the
- * ApplicationSubmissionContext
- *
- */
-@XmlRootElement(name = "am-black-listing-requests")
-@XmlAccessorType(XmlAccessType.FIELD)
-public class AMBlackListingRequestInfo {
-
-  @XmlElement(name = "am-black-listing-enabled")
-  boolean isAMBlackListingEnabled;
-
-  @XmlElement(name = "disable-failure-threshold")
-  float disableFailureThreshold;
-
-  public AMBlackListingRequestInfo() {
-  }
-
-  public boolean getAMBlackListingEnabled() {
-    return isAMBlackListingEnabled;
-  }
-
-  public void setAMBlackListingEnabled(boolean isAMBlackListingEnabled) {
-    this.isAMBlackListingEnabled = isAMBlackListingEnabled;
-  }
-
-  public float getBlackListingDisableFailureThreshold() {
-    return disableFailureThreshold;
-  }
-
-  public void setBlackListingDisableFailureThreshold(
-      float disableFailureThreshold) {
-    this.disableFailureThreshold = disableFailureThreshold;
-  }
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[29/51] [abbrv] hadoop git commit: YARN-3426. Add jdiff support to YARN. (vinodkv via wangda)

Posted by vv...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/03fc6b1b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.7.2.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.7.2.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.7.2.xml
new file mode 100644
index 0000000..f877336
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.7.2.xml
@@ -0,0 +1,3323 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Thu May 12 17:47:43 PDT 2016 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="hadoop-yarn-common 2.7.2"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.ExcludePrivateAnnotationsJDiffDoclet -docletpath /Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/hadoop-annotations.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/jdiff.jar -verbose -classpath /Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/classes:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-common-project/hadoop-common/target/hadoop-common-2.7.2.jar:/Users/vinodkv/.m2/repository/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/Users/vinodkv/.m2/repository/xmlenc/xmlenc/0.52/xmlenc-0.52.jar:/Users/vinodkv/.m2/repository/commons-httpclient/commons-httpclient/3.1/commons-httpclient-3.1.jar:/Users/vinodkv/.m2/repository/commons-net/commons-net/3.1/commons-
 net-3.1.jar:/Users/vinodkv/.m2/repository/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/Users/vinodkv/.m2/repository/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/Users/vinodkv/.m2/repository/javax/servlet/jsp/jsp-api/2.1/jsp-api-2.1.jar:/Users/vinodkv/.m2/repository/net/java/dev/jets3t/jets3t/0.9.0/jets3t-0.9.0.jar:/Users/vinodkv/.m2/repository/org/apache/httpcomponents/httpclient/4.2.5/httpclient-4.2.5.jar:/Users/vinodkv/.m2/repository/org/apache/httpcomponents/httpcore/4.2.5/httpcore-4.2.5.jar:/Users/vinodkv/.m2/repository/com/jamesmurty/utils/java-xmlbuilder/0.4/java-xmlbuilder-0.4.jar:/Users/vinodkv/.m2/repository/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/Users/vinodkv/.m2/repository/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/Users/vinodkv/.m2/repository/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/Users/vinodkv/.m2/repository/commons-beanutils/commons-beanutils
 -core/1.8.0/commons-beanutils-core-1.8.0.jar:/Users/vinodkv/.m2/repository/org/slf4j/slf4j-log4j12/1.7.10/slf4j-log4j12-1.7.10.jar:/Users/vinodkv/.m2/repository/org/apache/avro/avro/1.7.4/avro-1.7.4.jar:/Users/vinodkv/.m2/repository/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/Users/vinodkv/.m2/repository/org/xerial/snappy/snappy-java/1.0.4.1/snappy-java-1.0.4.1.jar:/Users/vinodkv/.m2/repository/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-common-project/hadoop-auth/target/hadoop-auth-2.7.2.jar:/Users/vinodkv/.m2/repository/org/apache/directory/server/apacheds-kerberos-codec/2.0.0-M15/apacheds-kerberos-codec-2.0.0-M15.jar:/Users/vinodkv/.m2/repository/org/apache/directory/server/apacheds-i18n/2.0.0-M15/apacheds-i18n-2.0.0-M15.jar:/Users/vinodkv/.m2/repository/org/apache/directory/api/api-asn1-api/1.0.0-M20/api-asn1-api-1.0.0-M20.jar:/Users/vinodkv/.m2/repository/org/apache/directory/api/api-util/1.0
 .0-M20/api-util-1.0.0-M20.jar:/Users/vinodkv/.m2/repository/org/apache/curator/curator-framework/2.7.1/curator-framework-2.7.1.jar:/Users/vinodkv/.m2/repository/com/jcraft/jsch/0.1.42/jsch-0.1.42.jar:/Users/vinodkv/.m2/repository/org/apache/curator/curator-client/2.7.1/curator-client-2.7.1.jar:/Users/vinodkv/.m2/repository/org/apache/curator/curator-recipes/2.7.1/curator-recipes-2.7.1.jar:/Users/vinodkv/.m2/repository/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/Users/vinodkv/.m2/repository/org/apache/htrace/htrace-core/3.1.0-incubating/htrace-core-3.1.0-incubating.jar:/Users/vinodkv/.m2/repository/org/apache/zookeeper/zookeeper/3.4.6/zookeeper-3.4.6.jar:/Users/vinodkv/.m2/repository/io/netty/netty/3.6.2.Final/netty-3.6.2.Final.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/hadoop-yarn-api-2.7.2.jar:/Users/vinodkv/.m2/repository/javax/xml/bind/jaxb-api/2.2.2/jaxb-api-2.2.2.jar:/Users/vinodkv/.m2/repo
 sitory/javax/xml/stream/stax-api/1.0-2/stax-api-1.0-2.jar:/Users/vinodkv/.m2/repository/javax/activation/activation/1.1/activation-1.1.jar:/Users/vinodkv/.m2/repository/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/Users/vinodkv/.m2/repository/org/tukaani/xz/1.0/xz-1.0.jar:/Users/vinodkv/.m2/repository/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/Users/vinodkv/.m2/repository/javax/servlet/servlet-api/2.5/servlet-api-2.5.jar:/Users/vinodkv/.m2/repository/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/Users/vinodkv/.m2/repository/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/jersey-core/1.9/jersey-core-1.9.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/jersey-client/1.9/jersey-client-1.9.jar:/Users/vinodkv/.m2/repository/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/Users/vinodkv/.m2/repository/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1
 .9.13.jar:/Users/vinodkv/.m2/repository/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/Users/vinodkv/.m2/repository/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/Users/vinodkv/.m2/repository/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/Users/vinodkv/.m2/repository/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/Users/vinodkv/.m2/repository/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/Users/vinodkv/.m2/repository/org/slf4j/slf4j-api/1.7.10/slf4j-api-1.7.10.jar:/Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-common-project/hadoop-annotations/target/hadoop-annotations-2.7.2.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_45.jdk/Contents/Home/lib/tools.jar:/Users/vinodkv/.m2/repository/com/google/inject/extensions/guice-servlet/3.0/guice-servlet-3.0.jar:/Users/vinodkv/.m2/repository/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar:/Users/vinodkv/.m2/repository/commons-io/commons-io/2.4/commons
 -io-2.4.jar:/Users/vinodkv/.m2/repository/com/google/inject/guice/3.0/guice-3.0.jar:/Users/vinodkv/.m2/repository/javax/inject/javax.inject/1/javax.inject-1.jar:/Users/vinodkv/.m2/repository/aopalliance/aopalliance/1.0/aopalliance-1.0.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/jersey-server/1.9/jersey-server-1.9.jar:/Users/vinodkv/.m2/repository/asm/asm/3.2/asm-3.2.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/jersey-json/1.9/jersey-json-1.9.jar:/Users/vinodkv/.m2/repository/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/Users/vinodkv/.m2/repository/com/sun/xml/bind/jaxb-impl/2.2.3-1/jaxb-impl-2.2.3-1.jar:/Users/vinodkv/.m2/repository/com/sun/jersey/contribs/jersey-guice/1.9/jersey-guice-1.9.jar:/Users/vinodkv/.m2/repository/log4j/log4j/1.2.17/log4j-1.2.17.jar -sourcepath /Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java -apidir /Users/vinodkv/Workspace/eclipse-workspace/apache-git/hadoop/hado
 op-yarn-project/hadoop-yarn/hadoop-yarn-common/target/site/jdiff/xml -apiname hadoop-yarn-common 2.7.2 -->
+<package name="org.apache.hadoop.yarn">
+  <!-- start class org.apache.hadoop.yarn.ContainerLogAppender -->
+  <class name="ContainerLogAppender" extends="org.apache.log4j.FileAppender"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.io.Flushable"/>
+    <constructor name="ContainerLogAppender"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="activateOptions"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="append"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="event" type="org.apache.log4j.spi.LoggingEvent"/>
+    </method>
+    <method name="flush"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="close"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getContainerLogDir" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Getter/Setter methods for log4j.]]>
+      </doc>
+    </method>
+    <method name="setContainerLogDir"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerLogDir" type="java.lang.String"/>
+    </method>
+    <method name="getContainerLogFile" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setContainerLogFile"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerLogFile" type="java.lang.String"/>
+    </method>
+    <method name="getTotalLogFileSize" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setTotalLogFileSize"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="logSize" type="long"/>
+    </method>
+    <doc>
+    <![CDATA[A simple log4j-appender for container's logs.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.ContainerLogAppender -->
+  <!-- start class org.apache.hadoop.yarn.ContainerRollingLogAppender -->
+  <class name="ContainerRollingLogAppender" extends="org.apache.log4j.RollingFileAppender"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.io.Flushable"/>
+    <constructor name="ContainerRollingLogAppender"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="activateOptions"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="flush"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getContainerLogDir" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Getter/Setter methods for log4j.]]>
+      </doc>
+    </method>
+    <method name="setContainerLogDir"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerLogDir" type="java.lang.String"/>
+    </method>
+    <method name="getContainerLogFile" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setContainerLogFile"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerLogFile" type="java.lang.String"/>
+    </method>
+    <doc>
+    <![CDATA[A simple log4j-appender for container's logs.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.ContainerRollingLogAppender -->
+  <!-- start class org.apache.hadoop.yarn.YarnUncaughtExceptionHandler -->
+  <class name="YarnUncaughtExceptionHandler" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.lang.Thread.UncaughtExceptionHandler"/>
+    <constructor name="YarnUncaughtExceptionHandler"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="uncaughtException"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="t" type="java.lang.Thread"/>
+      <param name="e" type="java.lang.Throwable"/>
+    </method>
+    <doc>
+    <![CDATA[This class is intended to be installed by calling
+ {@link Thread#setDefaultUncaughtExceptionHandler(UncaughtExceptionHandler)}
+ In the main entry point.  It is intended to try and cleanly shut down
+ programs using the Yarn Event framework.
+
+ Note: Right now it only will shut down the program if a Error is caught, but
+ not any other exception.  Anything else is just logged.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.YarnUncaughtExceptionHandler -->
+</package>
+<package name="org.apache.hadoop.yarn.api">
+</package>
+<package name="org.apache.hadoop.yarn.client">
+  <!-- start class org.apache.hadoop.yarn.client.AHSProxy -->
+  <class name="AHSProxy" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AHSProxy"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createAHSProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="ahsAddress" type="java.net.InetSocketAddress"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="rmAddress" type="java.net.InetSocketAddress"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.AHSProxy -->
+  <!-- start class org.apache.hadoop.yarn.client.ClientRMProxy -->
+  <class name="ClientRMProxy" extends="org.apache.hadoop.yarn.client.RMProxy"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="createRMProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="configuration" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Create a proxy to the ResourceManager for the specified protocol.
+ @param configuration Configuration with all the required information.
+ @param protocol Client protocol for which proxy is being requested.
+ @param <T> Type of proxy.
+ @return Proxy to the ResourceManager for the specified client protocol.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getRMDelegationTokenService" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <doc>
+      <![CDATA[Get the token service name to be used for RMDelegationToken. Depending
+ on whether HA is enabled or not, this method generates the appropriate
+ service name as a comma-separated list of service addresses.
+
+ @param conf Configuration corresponding to the cluster we need the
+             RMDelegationToken for
+ @return - Service name for RMDelegationToken]]>
+      </doc>
+    </method>
+    <method name="getAMRMTokenService" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getTokenService" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="address" type="java.lang.String"/>
+      <param name="defaultAddr" type="java.lang.String"/>
+      <param name="defaultPort" type="int"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.ClientRMProxy -->
+  <!-- start class org.apache.hadoop.yarn.client.NMProxy -->
+  <class name="NMProxy" extends="org.apache.hadoop.yarn.client.ServerProxy"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NMProxy"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createNMProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="ugi" type="org.apache.hadoop.security.UserGroupInformation"/>
+      <param name="rpc" type="org.apache.hadoop.yarn.ipc.YarnRPC"/>
+      <param name="serverAddress" type="java.net.InetSocketAddress"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.NMProxy -->
+  <!-- start class org.apache.hadoop.yarn.client.RMHAServiceTarget -->
+  <class name="RMHAServiceTarget" extends="org.apache.hadoop.ha.HAServiceTarget"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="RMHAServiceTarget" type="org.apache.hadoop.yarn.conf.YarnConfiguration"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </constructor>
+    <method name="getAddress" return="java.net.InetSocketAddress"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getZKFCAddress" return="java.net.InetSocketAddress"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getFencer" return="org.apache.hadoop.ha.NodeFencer"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="checkFencingConfigured"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="BadFencingConfigurationException" type="org.apache.hadoop.ha.BadFencingConfigurationException"/>
+    </method>
+    <method name="isAutoFailoverEnabled" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.RMHAServiceTarget -->
+  <!-- start class org.apache.hadoop.yarn.client.RMProxy -->
+  <class name="RMProxy" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="RMProxy"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createRMProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="This method is deprecated and is not used by YARN internally any more.
+ To create a proxy to the RM, use ClientRMProxy#createRMProxy or
+ ServerRMProxy#createRMProxy.
+
+ Create a proxy to the ResourceManager at the specified address.">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="rmAddress" type="java.net.InetSocketAddress"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[@deprecated
+ This method is deprecated and is not used by YARN internally any more.
+ To create a proxy to the RM, use ClientRMProxy#createRMProxy or
+ ServerRMProxy#createRMProxy.
+
+ Create a proxy to the ResourceManager at the specified address.
+
+ @param conf Configuration to generate retry policy
+ @param protocol Protocol for the proxy
+ @param rmAddress Address of the ResourceManager
+ @param <T> Type information of the proxy
+ @return Proxy to the RM
+ @throws IOException]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.RMProxy -->
+  <!-- start class org.apache.hadoop.yarn.client.ServerProxy -->
+  <class name="ServerProxy" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ServerProxy"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createRetryPolicy" return="org.apache.hadoop.io.retry.RetryPolicy"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="maxWaitTimeStr" type="java.lang.String"/>
+      <param name="defMaxWaitTime" type="long"/>
+      <param name="connectRetryIntervalStr" type="java.lang.String"/>
+      <param name="defRetryInterval" type="long"/>
+    </method>
+    <method name="createRetriableProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="user" type="org.apache.hadoop.security.UserGroupInformation"/>
+      <param name="rpc" type="org.apache.hadoop.yarn.ipc.YarnRPC"/>
+      <param name="serverAddress" type="java.net.InetSocketAddress"/>
+      <param name="retryPolicy" type="org.apache.hadoop.io.retry.RetryPolicy"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.ServerProxy -->
+</package>
+<package name="org.apache.hadoop.yarn.client.api">
+  <!-- start class org.apache.hadoop.yarn.client.api.TimelineClient -->
+  <class name="TimelineClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="TimelineClient" type="java.lang.String"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createTimelineClient" return="org.apache.hadoop.yarn.client.api.TimelineClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a timeline client. The current UGI when the user initialize the
+ client will be used to do the put and the delegation token operations. The
+ current user may use {@link UserGroupInformation#doAs} another user to
+ construct and initialize a timeline client if the following operations are
+ supposed to be conducted by that user.
+
+ @return a timeline client]]>
+      </doc>
+    </method>
+    <method name="putEntities" return="org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="entities" type="org.apache.hadoop.yarn.api.records.timeline.TimelineEntity[]"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Send the information of a number of conceptual entities to the timeline
+ server. It is a blocking API. The method will not return until it gets the
+ response from the timeline server.
+ </p>
+
+ @param entities
+          the collection of {@link TimelineEntity}
+ @return the error information if the sent entities are not correctly stored
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="putDomain"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="domain" type="org.apache.hadoop.yarn.api.records.timeline.TimelineDomain"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Send the information of a domain to the timeline server. It is a
+ blocking API. The method will not return until it gets the response from
+ the timeline server.
+ </p>
+
+ @param domain
+          an {@link TimelineDomain} object
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="getDelegationToken" return="org.apache.hadoop.security.token.Token"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="renewer" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a delegation token so as to be able to talk to the timeline server in a
+ secure way.
+ </p>
+
+ @param renewer
+          Address of the renewer who can renew these tokens when needed by
+          securely talking to the timeline server
+ @return a delegation token ({@link Token}) that can be used to talk to the
+         timeline server
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="renewDelegationToken" return="long"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="timelineDT" type="org.apache.hadoop.security.token.Token"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Renew a timeline delegation token.
+ </p>
+
+ @param timelineDT
+          the delegation token to renew
+ @return the new expiration time
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="cancelDelegationToken"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="timelineDT" type="org.apache.hadoop.security.token.Token"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Cancel a timeline delegation token.
+ </p>
+
+ @param timelineDT
+          the delegation token to cancel
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[A client library that can be used to post some information in terms of a
+ number of conceptual entities.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.TimelineClient -->
+</package>
+<package name="org.apache.hadoop.yarn.client.api.impl">
+</package>
+<package name="org.apache.hadoop.yarn.event">
+  <!-- start class org.apache.hadoop.yarn.event.AbstractEvent -->
+  <class name="AbstractEvent" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.event.Event"/>
+    <constructor name="AbstractEvent" type="TYPE"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AbstractEvent" type="TYPE, long"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getTimestamp" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getType" return="TYPE"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <doc>
+    <![CDATA[Parent class of all the events. All events extend this class.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.event.AbstractEvent -->
+  <!-- start class org.apache.hadoop.yarn.event.AsyncDispatcher -->
+  <class name="AsyncDispatcher" extends="org.apache.hadoop.service.AbstractService"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.event.Dispatcher"/>
+    <constructor name="AsyncDispatcher"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AsyncDispatcher" type="java.util.concurrent.BlockingQueue"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="serviceInit"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="serviceStart"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="setDrainEventsOnStop"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="serviceStop"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="dispatch"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="event" type="org.apache.hadoop.yarn.event.Event"/>
+    </method>
+    <method name="register"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="eventType" type="java.lang.Class"/>
+      <param name="handler" type="org.apache.hadoop.yarn.event.EventHandler"/>
+    </method>
+    <method name="getEventHandler" return="org.apache.hadoop.yarn.event.EventHandler"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="isEventThreadWaiting" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </method>
+    <method name="isDrained" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </method>
+    <field name="eventDispatchers" type="java.util.Map"
+      transient="false" volatile="false"
+      static="false" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[Dispatches {@link Event}s in a separate thread. Currently only single thread
+ does that. Potentially there could be multiple channels for each event type
+ class and a thread pool can be used to dispatch the events.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.event.AsyncDispatcher -->
+  <!-- start interface org.apache.hadoop.yarn.event.Dispatcher -->
+  <interface name="Dispatcher"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getEventHandler" return="org.apache.hadoop.yarn.event.EventHandler"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="register"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="eventType" type="java.lang.Class"/>
+      <param name="handler" type="org.apache.hadoop.yarn.event.EventHandler"/>
+    </method>
+    <field name="DISPATCHER_EXIT_ON_ERROR_KEY" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="DEFAULT_DISPATCHER_EXIT_ON_ERROR" type="boolean"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[Event Dispatcher interface. It dispatches events to registered
+ event handlers based on event types.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.event.Dispatcher -->
+  <!-- start interface org.apache.hadoop.yarn.event.Event -->
+  <interface name="Event"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getType" return="TYPE"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getTimestamp" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <doc>
+    <![CDATA[Interface defining events api.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.event.Event -->
+  <!-- start interface org.apache.hadoop.yarn.event.EventHandler -->
+  <interface name="EventHandler"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="handle"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="event" type="T"/>
+    </method>
+    <doc>
+    <![CDATA[Interface for handling events of type T
+
+ @param <T> parameterized event of type T]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.event.EventHandler -->
+</package>
+<package name="org.apache.hadoop.yarn.factories">
+</package>
+<package name="org.apache.hadoop.yarn.factory.providers">
+</package>
+<package name="org.apache.hadoop.yarn.logaggregation">
+  <!-- start class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat -->
+  <class name="AggregatedLogFormat" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AggregatedLogFormat"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat -->
+  <!-- start class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey -->
+  <class name="AggregatedLogFormat.LogKey" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.io.Writable"/>
+    <constructor name="AggregatedLogFormat.LogKey"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AggregatedLogFormat.LogKey" type="org.apache.hadoop.yarn.api.records.ContainerId"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AggregatedLogFormat.LogKey" type="java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="obj" type="java.lang.Object"/>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey -->
+  <!-- start class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogReader -->
+  <class name="AggregatedLogFormat.LogReader" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AggregatedLogFormat.LogReader" type="org.apache.hadoop.conf.Configuration, org.apache.hadoop.fs.Path"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </constructor>
+    <method name="getApplicationOwner" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Returns the owner of the application.
+
+ @return the application owner.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAcls" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Returns ACLs for the application. An empty map is returned if no ACLs are
+ found.
+
+ @return a map of the Application ACLs.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="next" return="java.io.DataInputStream"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="key" type="org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Read the next key and return the value-stream.
+
+ @param key
+ @return the valueStream if there are more keys or null otherwise.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="readAcontainerLogs"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="writer" type="java.io.Writer"/>
+      <param name="logUploadedTime" type="long"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Writes all logs for a single container to the provided writer.
+ @param valueStream
+ @param writer
+ @param logUploadedTime
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="readAcontainerLogs"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="writer" type="java.io.Writer"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Writes all logs for a single container to the provided writer.
+ @param valueStream
+ @param writer
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="readAContainerLogsForALogType"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="out" type="java.io.PrintStream"/>
+      <param name="logUploadedTime" type="long"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Keep calling this till you get a {@link EOFException} for getting logs of
+ all types for a single container.
+
+ @param valueStream
+ @param out
+ @param logUploadedTime
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="readAContainerLogsForALogType"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="out" type="java.io.PrintStream"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Keep calling this till you get a {@link EOFException} for getting logs of
+ all types for a single container.
+
+ @param valueStream
+ @param out
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="close"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogReader -->
+  <!-- start class org.apache.hadoop.yarn.logaggregation.LogCLIHelpers -->
+  <class name="LogCLIHelpers" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.conf.Configurable"/>
+    <constructor name="LogCLIHelpers"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="setConf"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getConf" return="org.apache.hadoop.conf.Configuration"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.logaggregation.LogCLIHelpers -->
+</package>
+<package name="org.apache.hadoop.yarn.nodelabels">
+  <!-- start class org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager -->
+  <class name="CommonNodeLabelsManager" extends="org.apache.hadoop.service.AbstractService"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="CommonNodeLabelsManager"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="handleStoreEvent"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="event" type="org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEvent"/>
+    </method>
+    <method name="initDispatcher"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="serviceInit"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="initNodeLabelStore"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="startDispatcher"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </method>
+    <method name="serviceStart"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="stopDispatcher"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </method>
+    <method name="serviceStop"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="addToCluserNodeLabels"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="labels" type="java.util.Set"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Add multiple node labels to repository
+
+ @param labels
+          new node labels added]]>
+      </doc>
+    </method>
+    <method name="checkAddLabelsToNode"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="addedLabelsToNode" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="addLabelsToNode"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="addedLabelsToNode" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[add more labels to nodes
+
+ @param addedLabelsToNode node {@literal ->} labels map]]>
+      </doc>
+    </method>
+    <method name="checkRemoveFromClusterNodeLabels"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="labelsToRemove" type="java.util.Collection"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="internalRemoveFromClusterNodeLabels"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="labelsToRemove" type="java.util.Collection"/>
+    </method>
+    <method name="removeFromClusterNodeLabels"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="labelsToRemove" type="java.util.Collection"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Remove multiple node labels from repository
+
+ @param labelsToRemove
+          node labels to remove
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="checkRemoveLabelsFromNode"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="removeLabelsFromNode" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="removeNodeFromLabels"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="node" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <param name="labels" type="java.util.Set"/>
+    </method>
+    <method name="internalUpdateLabelsOnNodes"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="nodeToLabels" type="java.util.Map"/>
+      <param name="op" type="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.NodeLabelUpdateOperation"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="removeLabelsFromNode"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="removeLabelsFromNode" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[remove labels from nodes, labels being removed most be contained by these
+ nodes
+
+ @param removeLabelsFromNode node {@literal ->} labels map]]>
+      </doc>
+    </method>
+    <method name="checkReplaceLabelsOnNode"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="replaceLabelsToNode" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="replaceLabelsOnNode"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="replaceLabelsToNode" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[replace labels to nodes
+
+ @param replaceLabelsToNode node {@literal ->} labels map]]>
+      </doc>
+    </method>
+    <method name="getNodeLabels" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get mapping of nodes to labels
+
+ @return nodes to labels map]]>
+      </doc>
+    </method>
+    <method name="getLabelsToNodes" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get mapping of labels to nodes for all the labels.
+
+ @return labels to nodes map]]>
+      </doc>
+    </method>
+    <method name="getLabelsToNodes" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="labels" type="java.util.Set"/>
+      <doc>
+      <![CDATA[Get mapping of labels to nodes for specified set of labels.
+
+ @param labels set of labels for which labels to nodes mapping will be
+        returned.
+ @return labels to nodes map]]>
+      </doc>
+    </method>
+    <method name="getClusterNodeLabels" return="java.util.Set"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get existing valid labels in repository
+
+ @return existing valid labels in repository]]>
+      </doc>
+    </method>
+    <method name="normalizeLabel" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="label" type="java.lang.String"/>
+    </method>
+    <method name="getNMInNodeSet" return="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Node"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+    </method>
+    <method name="getNMInNodeSet" return="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Node"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <param name="map" type="java.util.Map"/>
+    </method>
+    <method name="getNMInNodeSet" return="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Node"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <param name="map" type="java.util.Map"/>
+      <param name="checkRunning" type="boolean"/>
+    </method>
+    <method name="getLabelsByNode" return="java.util.Set"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+    </method>
+    <method name="getLabelsByNode" return="java.util.Set"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <param name="map" type="java.util.Map"/>
+    </method>
+    <method name="createNodeIfNonExisted"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="createHostIfNonExisted"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="hostName" type="java.lang.String"/>
+    </method>
+    <method name="normalizeNodeIdToLabels" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="nodeIdToLabels" type="java.util.Map"/>
+    </method>
+    <field name="LOG" type="org.apache.commons.logging.Log"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="EMPTY_STRING_SET" type="java.util.Set"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="ANY" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="ACCESS_ANY_LABEL_SET" type="java.util.Set"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="WILDCARD_PORT" type="int"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="NODE_LABELS_NOT_ENABLED_ERR" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Error messages]]>
+      </doc>
+    </field>
+    <field name="NO_LABEL" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[If a user doesn't specify label of a queue or node, it belongs
+ DEFAULT_LABEL]]>
+      </doc>
+    </field>
+    <field name="dispatcher" type="org.apache.hadoop.yarn.event.Dispatcher"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="labelCollections" type="java.util.concurrent.ConcurrentMap"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="nodeCollections" type="java.util.concurrent.ConcurrentMap"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="readLock" type="java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock"
+      transient="false" volatile="false"
+      static="false" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="writeLock" type="java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock"
+      transient="false" volatile="false"
+      static="false" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="store" type="org.apache.hadoop.yarn.nodelabels.NodeLabelsStore"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Host -->
+  <class name="CommonNodeLabelsManager.Host" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="protected"
+    deprecated="not deprecated">
+    <constructor name="CommonNodeLabelsManager.Host"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="copy" return="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Host"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="labels" type="java.util.Set"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="nms" type="java.util.Map"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[A <code>Host</code> can have multiple <code>Node</code>s]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Host -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Node -->
+  <class name="CommonNodeLabelsManager.Node" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="protected"
+    deprecated="not deprecated">
+    <constructor name="CommonNodeLabelsManager.Node" type="org.apache.hadoop.yarn.api.records.NodeId"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="copy" return="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Node"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="labels" type="java.util.Set"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="resource" type="org.apache.hadoop.yarn.api.records.Resource"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="running" type="boolean"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.Node -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore -->
+  <class name="FileSystemNodeLabelsStore" extends="org.apache.hadoop.yarn.nodelabels.NodeLabelsStore"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="FileSystemNodeLabelsStore" type="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="init"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="close"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="updateNodeToLabelsMappings"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeToLabels" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="storeNewClusterNodeLabels"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="labels" type="java.util.Set"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="removeClusterNodeLabels"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="labels" type="java.util.Collection"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="recover"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <field name="LOG" type="org.apache.commons.logging.Log"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="DEFAULT_DIR_NAME" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="MIRROR_FILENAME" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="EDITLOG_FILENAME" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore.SerializedLogType -->
+  <class name="FileSystemNodeLabelsStore.SerializedLogType" extends="java.lang.Enum"
+    abstract="false"
+    static="true" final="true" visibility="protected"
+    deprecated="not deprecated">
+    <method name="values" return="org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore.SerializedLogType[]"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="valueOf" return="org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore.SerializedLogType"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore.SerializedLogType -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.NodeLabel -->
+  <class name="NodeLabel" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.lang.Comparable"/>
+    <constructor name="NodeLabel" type="java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="NodeLabel" type="java.lang.String, org.apache.hadoop.yarn.api.records.Resource, int"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="addNodeId"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="node" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+    </method>
+    <method name="removeNodeId"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="node" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+    </method>
+    <method name="getAssociatedNodeIds" return="java.util.Set"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="addNode"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeRes" type="org.apache.hadoop.yarn.api.records.Resource"/>
+    </method>
+    <method name="removeNode"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeRes" type="org.apache.hadoop.yarn.api.records.Resource"/>
+    </method>
+    <method name="getResource" return="org.apache.hadoop.yarn.api.records.Resource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getNumActiveNMs" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getLabelName" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getCopy" return="org.apache.hadoop.yarn.nodelabels.NodeLabel"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="compareTo" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="o" type="org.apache.hadoop.yarn.nodelabels.NodeLabel"/>
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="obj" type="java.lang.Object"/>
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.NodeLabel -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.NodeLabelsStore -->
+  <class name="NodeLabelsStore" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.io.Closeable"/>
+    <constructor name="NodeLabelsStore" type="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="updateNodeToLabelsMappings"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeToLabels" type="java.util.Map"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Store node {@literal ->} label]]>
+      </doc>
+    </method>
+    <method name="storeNewClusterNodeLabels"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="label" type="java.util.Set"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Store new labels]]>
+      </doc>
+    </method>
+    <method name="removeClusterNodeLabels"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="labels" type="java.util.Collection"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Remove labels]]>
+      </doc>
+    </method>
+    <method name="recover"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Recover labels and node to labels mappings from store]]>
+      </doc>
+    </method>
+    <method name="init"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="getNodeLabelsManager" return="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="mgr" type="org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager"
+      transient="false" volatile="false"
+      static="false" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.NodeLabelsStore -->
+</package>
+<package name="org.apache.hadoop.yarn.nodelabels.event">
+  <!-- start class org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEvent -->
+  <class name="NodeLabelsStoreEvent" extends="org.apache.hadoop.yarn.event.AbstractEvent"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NodeLabelsStoreEvent" type="org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEventType"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEvent -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEventType -->
+  <class name="NodeLabelsStoreEventType" extends="java.lang.Enum"
+    abstract="false"
+    static="false" final="true" visibility="public"
+    deprecated="not deprecated">
+    <method name="values" return="org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEventType[]"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="valueOf" return="org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEventType"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEventType -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.event.RemoveClusterNodeLabels -->
+  <class name="RemoveClusterNodeLabels" extends="org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEvent"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="RemoveClusterNodeLabels" type="java.util.Collection"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getLabels" return="java.util.Collection"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.event.RemoveClusterNodeLabels -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.event.StoreNewClusterNodeLabels -->
+  <class name="StoreNewClusterNodeLabels" extends="org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEvent"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="StoreNewClusterNodeLabels" type="java.util.Set"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getLabels" return="java.util.Set"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.event.StoreNewClusterNodeLabels -->
+  <!-- start class org.apache.hadoop.yarn.nodelabels.event.UpdateNodeToLabelsMappingsEvent -->
+  <class name="UpdateNodeToLabelsMappingsEvent" extends="org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEvent"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="UpdateNodeToLabelsMappingsEvent" type="java.util.Map"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getNodeToLabels" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.nodelabels.event.UpdateNodeToLabelsMappingsEvent -->
+</package>
+<package name="org.apache.hadoop.yarn.security">
+  <!-- start class org.apache.hadoop.yarn.security.AMRMTokenIdentifier -->
+  <class name="AMRMTokenIdentifier" extends="org.apache.hadoop.security.token.TokenIdentifier"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AMRMTokenIdentifier"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AMRMTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId, int"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="write"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="out" type="java.io.DataOutput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="readFields"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.DataInput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getKind" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getUser" return="org.apache.hadoop.security.UserGroupInformation"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getKeyId" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos.AMRMTokenIdentifierProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="other" type="java.lang.Object"/>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="KIND_NAME" type="org.apache.hadoop.io.Text"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[AMRMTokenIdentifier is the TokenIdentifier to be used by
+ ApplicationMasters to authenticate to the ResourceManager.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.AMRMTokenIdentifier -->
+  <!-- start class org.apache.hadoop.yarn.security.AMRMTokenSelector -->
+  <class name="AMRMTokenSelector" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.security.token.TokenSelector"/>
+    <constructor name="AMRMTokenSelector"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="selectToken" return="org.apache.hadoop.security.token.Token"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="service" type="org.apache.hadoop.io.Text"/>
+      <param name="tokens" type="java.util.Collection"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.AMRMTokenSelector -->
+  <!-- start class org.apache.hadoop.yarn.security.ContainerManagerSecurityInfo -->
+  <class name="ContainerManagerSecurityInfo" extends="org.apache.hadoop.security.SecurityInfo"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ContainerManagerSecurityInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getKerberosInfo" return="org.apache.hadoop.security.KerberosInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getTokenInfo" return="org.apache.hadoop.security.token.TokenInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.ContainerManagerSecurityInfo -->
+  <!-- start class org.apache.hadoop.yarn.security.ContainerTokenIdentifier -->
+  <class name="ContainerTokenIdentifier" extends="org.apache.hadoop.security.token.TokenIdentifier"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ContainerTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ContainerId, java.lang.String, java.lang.String, org.apache.hadoop.yarn.api.records.Resource, long, int, long, org.apache.hadoop.yarn.api.records.Priority, long"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="ContainerTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ContainerId, java.lang.String, java.lang.String, org.apache.hadoop.yarn.api.records.Resource, long, int, long, org.apache.hadoop.yarn.api.records.Priority, long, org.apache.hadoop.yarn.api.records.LogAggregationContext"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="ContainerTokenIdentifier"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Default constructor needed by RPC layer/SecretManager.]]>
+      </doc>
+    </constructor>
+    <method name="getContainerID" return="org.apache.hadoop.yarn.api.records.ContainerId"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getApplicationSubmitter" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getNmHostAddress" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getResource" return="org.apache.hadoop.yarn.api.records.Resource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getExpiryTimeStamp" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getMasterKeyId" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getPriority" return="org.apache.hadoop.yarn.api.records.Priority"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getCreationTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>


<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[22/51] [abbrv] hadoop git commit: HADOOP-12666. Support Microsoft Azure Data Lake - as a file system in Hadoop. Contributed by Vishwajeet Dusane.

Posted by vv...@apache.org.
HADOOP-12666. Support Microsoft Azure Data Lake - as a file system in Hadoop. Contributed by Vishwajeet Dusane.


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

Branch: refs/heads/YARN-3926
Commit: 9581fb715cbc8a6ad28566e83c6d0242a7306688
Parents: e383b73
Author: Chris Nauroth <cn...@apache.org>
Authored: Thu Jun 9 14:33:31 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Thu Jun 9 14:33:31 2016 -0700

----------------------------------------------------------------------
 .../src/main/resources/core-default.xml         |   60 +
 .../conf/TestCommonConfigurationFields.java     |    6 +
 hadoop-project/src/site/site.xml                |    2 +
 .../dev-support/findbugs-exclude.xml            |   24 +
 hadoop-tools/hadoop-azure-datalake/pom.xml      |  180 +++
 .../main/java/org/apache/hadoop/fs/adl/Adl.java |   52 +
 .../org/apache/hadoop/fs/adl/AdlFileSystem.java |   41 +
 ...hedRefreshTokenBasedAccessTokenProvider.java |  135 +++
 .../hadoop/fs/adl/oauth2/package-info.java      |   23 +
 .../org/apache/hadoop/fs/adl/package-info.java  |   23 +
 .../org/apache/hadoop/hdfs/web/ADLConfKeys.java |   61 +
 .../apache/hadoop/hdfs/web/BufferManager.java   |  180 +++
 .../web/PrivateAzureDataLakeFileSystem.java     | 1108 ++++++++++++++++++
 ...hedRefreshTokenBasedAccessTokenProvider.java |   37 +
 .../hadoop/hdfs/web/oauth2/package-info.java    |   24 +
 .../apache/hadoop/hdfs/web/package-info.java    |   25 +
 .../hadoop/hdfs/web/resources/ADLFlush.java     |   49 +
 .../hdfs/web/resources/ADLGetOpParam.java       |   96 ++
 .../hdfs/web/resources/ADLPostOpParam.java      |   97 ++
 .../hdfs/web/resources/ADLPutOpParam.java       |   94 ++
 .../hdfs/web/resources/ADLVersionInfo.java      |   51 +
 .../web/resources/AppendADLNoRedirectParam.java |   45 +
 .../web/resources/CreateADLNoRedirectParam.java |   44 +
 .../hadoop/hdfs/web/resources/LeaseParam.java   |   53 +
 .../web/resources/ReadADLNoRedirectParam.java   |   44 +
 .../hadoop/hdfs/web/resources/package-info.java |   27 +
 .../src/site/markdown/index.md                  |  219 ++++
 ...hedRefreshTokenBasedAccessTokenProvider.java |  147 +++
 hadoop-tools/hadoop-tools-dist/pom.xml          |    6 +
 hadoop-tools/pom.xml                            |    1 +
 30 files changed, 2954 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 39b7132..f1d77dd 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -2213,4 +2213,64 @@
       needs to be specified in net.topology.script.file.name.
     </description>
   </property>
+
+
+  <!-- Azure Data Lake File System Configurations -->
+
+  <property>
+    <name>adl.feature.override.readahead</name>
+    <value>true</value>
+    <description>
+      Enables read aheads in the ADL client, the feature is used to
+      improve read throughput.
+      This works in conjunction with the value set in
+      adl.feature.override.readahead.max.buffersize.
+      When set to false the read ahead feature is turned off.
+      Default : True if not configured.
+    </description>
+  </property>
+
+  <property>
+    <name>adl.feature.override.readahead.max.buffersize</name>
+    <value>8388608</value>
+    <description>
+      Define maximum buffer size to cache read ahead data, this is
+      allocated per process to
+      cache read ahead data. Applicable only when
+      adl.feature.override.readahead is set to true.
+      Default : 8388608 Byte i.e. 8MB if not configured.
+    </description>
+  </property>
+
+  <property>
+    <name>adl.feature.override.readahead.max.concurrent.connection</name>
+    <value>2</value>
+    <description>
+      Define maximum concurrent connection can be established to
+      read ahead. If the data size is less than 4MB then only 1 read n/w
+      connection
+      is set. If the data size is less than 4MB but less than 8MB then 2 read
+      n/w connection
+      is set. Data greater than 8MB then value set under the property would
+      take
+      effect. Applicable only when adl.feature.override.readahead is set
+      to true and buffer size is greater than 8MB.
+      It is recommended to reset this property if the
+      adl.feature.override.readahead.max.buffersize
+      is less than 8MB to gain performance. Application has to consider
+      throttling limit for the account as well before configuring large
+      buffer size.
+    </description>
+  </property>
+
+  <property>
+    <name>fs.adl.impl</name>
+    <value>org.apache.hadoop.fs.adl.AdlFileSystem</value>
+  </property>
+
+  <property>
+    <name>fs.AbstractFileSystem.adl.impl</name>
+    <value>org.apache.hadoop.fs.adl.Adl</value>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
index 90f7514..020474f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
@@ -102,6 +102,12 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase {
     xmlPrefixToSkipCompare.add("s3.");
     xmlPrefixToSkipCompare.add("s3native.");
 
+    // ADL properties are in a different subtree
+    // - org.apache.hadoop.hdfs.web.ADLConfKeys
+    xmlPrefixToSkipCompare.add("adl.");
+    xmlPropsToSkipCompare.add("fs.adl.impl");
+    xmlPropsToSkipCompare.add("fs.AbstractFileSystem.adl.impl");
+
     // Deprecated properties.  These should eventually be removed from the
     // class.
     configurationPropsToSkipCompare

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-project/src/site/site.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index f9f4726..a89a220 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -146,6 +146,8 @@
     <menu name="Hadoop Compatible File Systems" inherit="top">
       <item name="Amazon S3" href="hadoop-aws/tools/hadoop-aws/index.html"/>
       <item name="Azure Blob Storage" href="hadoop-azure/index.html"/>
+      <item name="Azure Data Lake Storage"
+            href="hadoop-azure-datalake/index.html"/>
       <item name="OpenStack Swift" href="hadoop-openstack/index.html"/>
     </menu>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-azure-datalake/dev-support/findbugs-exclude.xml
new file mode 100644
index 0000000..4fd36ef
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/dev-support/findbugs-exclude.xml
@@ -0,0 +1,24 @@
+<!--
+   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.
+-->
+<FindBugsFilter>
+    <!-- Buffer object is accessed withing trusted code and intentionally assigned instead of array copy -->
+    <Match>
+        <Class name="org.apache.hadoop.hdfs.web.PrivateAzureDataLakeFileSystem$BatchAppendOutputStream$CommitTask"/>
+        <Bug pattern="EI_EXPOSE_REP2"/>
+        <Priority value="2"/>
+    </Match>
+</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/pom.xml b/hadoop-tools/hadoop-azure-datalake/pom.xml
new file mode 100644
index 0000000..a4b1fe1
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/pom.xml
@@ -0,0 +1,180 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>3.0.0-alpha1-SNAPSHOT</version>
+    <relativePath>../../hadoop-project</relativePath>
+  </parent>
+  <groupId>org.apache.hadoop</groupId>
+  <artifactId>hadoop-azure-datalake</artifactId>
+  <name>Apache Hadoop Azure Data Lake support</name>
+  <description>
+    This module contains code to support integration with Azure Data Lake.
+  </description>
+  <packaging>jar</packaging>
+  <properties>
+    <okHttpVersion>2.4.0</okHttpVersion>
+    <minimalJsonVersion>0.9.1</minimalJsonVersion>
+    <file.encoding>UTF-8</file.encoding>
+    <downloadSources>true</downloadSources>
+  </properties>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <findbugsXmlOutput>true</findbugsXmlOutput>
+          <xmlOutput>true</xmlOutput>
+          <excludeFilterFile>
+            ${basedir}/dev-support/findbugs-exclude.xml
+          </excludeFilterFile>
+          <effort>Max</effort>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-project-info-reports-plugin</artifactId>
+
+        <configuration>
+          <dependencyDetailsEnabled>false</dependencyDetailsEnabled>
+          <dependencyLocationsEnabled>false
+          </dependencyLocationsEnabled>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>deplist</id>
+            <phase>compile</phase>
+            <goals>
+              <goal>list</goal>
+            </goals>
+            <configuration>
+              <!-- build a shellprofile -->
+              <outputFile>${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-optional.txt</outputFile>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+
+
+    <!--
+       The following is to suppress a m2e warning in eclipse
+       (m2e doesn't know how to handle maven-enforcer:enforce, so we have to tell m2e to ignore it)
+       see: http://stackoverflow.com/questions/13040788/how-to-elimate-the-maven-enforcer-plugin-goal-enforce-is-ignored-by-m2e-wa
+    -->
+    <pluginManagement>
+      <plugins>
+        <plugin>
+          <groupId>org.eclipse.m2e</groupId>
+          <artifactId>lifecycle-mapping</artifactId>
+          <version>1.0.0</version>
+          <configuration>
+            <lifecycleMappingMetadata>
+              <pluginExecutions>
+                <pluginExecution>
+                  <pluginExecutionFilter>
+                    <groupId>org.apache.maven.plugins
+                    </groupId>
+                    <artifactId>maven-enforcer-plugin
+                    </artifactId>
+                    <versionRange>[1.0.0,)</versionRange>
+                    <goals>
+                      <goal>enforce</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <ignore/>
+                  </action>
+                </pluginExecution>
+              </pluginExecutions>
+            </lifecycleMappingMetadata>
+          </configuration>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+      <exclusions>
+        <exclusion>
+          <artifactId>servlet-api</artifactId>
+          <groupId>javax.servlet</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs-client</artifactId>
+    </dependency>
+    <dependency>
+    <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+  </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.eclipsesource.minimal-json</groupId>
+      <artifactId>minimal-json</artifactId>
+      <version>0.9.1</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>com.squareup.okhttp</groupId>
+      <artifactId>mockwebserver</artifactId>
+      <version>2.4.0</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/Adl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/Adl.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/Adl.java
new file mode 100644
index 0000000..4642d6b
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/Adl.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegateToFileSystem;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+/**
+ * Expose adl:// scheme to access ADL file system.
+ */
+public class Adl extends DelegateToFileSystem {
+
+  Adl(URI theUri, Configuration conf) throws IOException, URISyntaxException {
+    super(theUri, createDataLakeFileSystem(conf), conf, AdlFileSystem.SCHEME,
+        false);
+  }
+
+  private static AdlFileSystem createDataLakeFileSystem(Configuration conf) {
+    AdlFileSystem fs = new AdlFileSystem();
+    fs.setConf(conf);
+    return fs;
+  }
+
+  /**
+   * @return Default port for ADL File system to communicate
+   */
+  @Override
+  public final int getUriDefaultPort() {
+    return AdlFileSystem.DEFAULT_PORT;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
new file mode 100644
index 0000000..11e1e0b
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
@@ -0,0 +1,41 @@
+/*
+ * 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.adl;
+
+import org.apache.hadoop.hdfs.web.PrivateAzureDataLakeFileSystem;
+
+/**
+ * Expose adl:// scheme to access ADL file system.
+ */
+public class AdlFileSystem extends PrivateAzureDataLakeFileSystem {
+
+  public static final String SCHEME = "adl";
+  public static final int DEFAULT_PORT = 443;
+
+  @Override
+  public String getScheme() {
+    return SCHEME;
+  }
+
+  @Override
+  public int getDefaultPort() {
+    return DEFAULT_PORT;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/oauth2/CachedRefreshTokenBasedAccessTokenProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/oauth2/CachedRefreshTokenBasedAccessTokenProvider.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/oauth2/CachedRefreshTokenBasedAccessTokenProvider.java
new file mode 100644
index 0000000..b7f3b00
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/oauth2/CachedRefreshTokenBasedAccessTokenProvider.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.adl.oauth2;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.LinkedHashMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.web.oauth2.AccessTokenProvider;
+import org.apache.hadoop.hdfs.web.oauth2.ConfRefreshTokenBasedAccessTokenProvider;
+import org.apache.hadoop.hdfs.web.oauth2.PrivateCachedRefreshTokenBasedAccessTokenProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY;
+import static org.apache.hadoop.hdfs.web.oauth2.ConfRefreshTokenBasedAccessTokenProvider.OAUTH_REFRESH_TOKEN_KEY;
+
+/**
+ * Share refresh tokens across all ADLS instances with a common client ID. The
+ * {@link AccessTokenProvider} can be shared across multiple instances,
+ * amortizing the cost of refreshing tokens.
+ */
+public class CachedRefreshTokenBasedAccessTokenProvider
+    extends PrivateCachedRefreshTokenBasedAccessTokenProvider {
+
+  public static final String FORCE_REFRESH = "adl.force.token.refresh";
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CachedRefreshTokenBasedAccessTokenProvider.class);
+
+  /** Limit size of provider cache. */
+  static final int MAX_PROVIDERS = 10;
+  @SuppressWarnings("serial")
+  private static final Map<String, AccessTokenProvider> CACHE =
+      new LinkedHashMap<String, AccessTokenProvider>() {
+        @Override
+        public boolean removeEldestEntry(
+            Map.Entry<String, AccessTokenProvider> e) {
+          return size() > MAX_PROVIDERS;
+        }
+      };
+
+  private AccessTokenProvider instance = null;
+
+  /**
+   * Create handle for cached instance.
+   */
+  public CachedRefreshTokenBasedAccessTokenProvider() {
+  }
+
+  /**
+   * Gets the access token from internally cached
+   * ConfRefreshTokenBasedAccessTokenProvider instance.
+   *
+   * @return Valid OAuth2 access token for the user.
+   * @throws IOException when system error, internal server error or user error
+   */
+  @Override
+  public synchronized String getAccessToken() throws IOException {
+    return instance.getAccessToken();
+  }
+
+  /**
+   * @return A cached Configuration consistent with the parameters of this
+   * instance.
+   */
+  @Override
+  public synchronized Configuration getConf() {
+    return instance.getConf();
+  }
+
+  /**
+   * Configure cached instance. Note that the Configuration instance returned
+   * from subsequent calls to {@link #getConf() getConf} may be from a
+   * previous, cached entry.
+   * @param conf Configuration instance
+   */
+  @Override
+  public synchronized void setConf(Configuration conf) {
+    String id = conf.get(OAUTH_CLIENT_ID_KEY);
+    if (null == id) {
+      throw new IllegalArgumentException("Missing client ID");
+    }
+    synchronized (CACHE) {
+      instance = CACHE.get(id);
+      if (null == instance
+          || conf.getBoolean(FORCE_REFRESH, false)
+          || replace(instance, conf)) {
+        instance = newInstance();
+        // clone configuration
+        instance.setConf(new Configuration(conf));
+        CACHE.put(id, instance);
+        LOG.debug("Created new client {}", id);
+      }
+    }
+  }
+
+  AccessTokenProvider newInstance() {
+    return new ConfRefreshTokenBasedAccessTokenProvider();
+  }
+
+  private static boolean replace(AccessTokenProvider cached, Configuration c2) {
+    // ConfRefreshTokenBasedAccessTokenProvider::setConf asserts !null
+    final Configuration c1 = cached.getConf();
+    for (String key : new String[] {
+        OAUTH_REFRESH_TOKEN_KEY, OAUTH_REFRESH_URL_KEY }) {
+      if (!c1.get(key).equals(c2.get(key))) {
+        // replace cached instance for this clientID
+        return true;
+      }
+    }
+    return false;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/oauth2/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/oauth2/package-info.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/oauth2/package-info.java
new file mode 100644
index 0000000..b444984
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/oauth2/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ *
+ */
+
+/**
+ * public interface to expose OAuth2 authentication related features.
+ */
+package org.apache.hadoop.fs.adl.oauth2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/package-info.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/package-info.java
new file mode 100644
index 0000000..98e6a77
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ *
+ */
+
+/**
+ * Supporting classes for metrics instrumentation.
+ */
+package org.apache.hadoop.fs.adl;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/ADLConfKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/ADLConfKeys.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/ADLConfKeys.java
new file mode 100644
index 0000000..a7f932f
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/ADLConfKeys.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.hdfs.web;
+
+/**
+ * Constants.
+ */
+public final class ADLConfKeys {
+  public static final String
+      ADL_FEATURE_CONCURRENT_READ_AHEAD_MAX_CONCURRENT_CONN =
+      "adl.feature.override.readahead.max.concurrent.connection";
+  public static final int
+      ADL_FEATURE_CONCURRENT_READ_AHEAD_MAX_CONCURRENT_CONN_DEFAULT = 2;
+  public static final String ADL_WEBSDK_VERSION_KEY = "ADLFeatureSet";
+  static final String ADL_DEBUG_OVERRIDE_LOCAL_USER_AS_OWNER =
+      "adl.debug.override.localuserasfileowner";
+  static final boolean ADL_DEBUG_SET_LOCAL_USER_AS_OWNER_DEFAULT = false;
+  static final String ADL_FEATURE_REDIRECT_OFF =
+      "adl.feature.override.redirection.off";
+  static final boolean ADL_FEATURE_REDIRECT_OFF_DEFAULT = true;
+  static final String ADL_FEATURE_GET_BLOCK_LOCATION_LOCALLY_BUNDLED =
+      "adl.feature.override.getblocklocation.locally.bundled";
+  static final boolean ADL_FEATURE_GET_BLOCK_LOCATION_LOCALLY_BUNDLED_DEFAULT
+      = true;
+  static final String ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD =
+      "adl.feature.override.readahead";
+  static final boolean ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD_DEFAULT =
+      true;
+  static final String ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD_BUFFER_SIZE =
+      "adl.feature.override.readahead.max.buffersize";
+
+  static final int KB = 1024;
+  static final int MB = KB * KB;
+  static final int DEFAULT_BLOCK_SIZE = 4 * MB;
+  static final int DEFAULT_EXTENT_SIZE = 256 * MB;
+  static final int DEFAULT_TIMEOUT_IN_SECONDS = 120;
+  static final int
+      ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD_BUFFER_SIZE_DEFAULT =
+      8 * MB;
+
+  private ADLConfKeys() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/BufferManager.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/BufferManager.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/BufferManager.java
new file mode 100644
index 0000000..350c6e7
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/BufferManager.java
@@ -0,0 +1,180 @@
+/*
+ * 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.web;
+
+/**
+ * Responsible for holding buffered data in the process. Hold only 1 and only
+ * 1 buffer block in the memory. Buffer block
+ * information is for the given file and the offset from the which the block
+ * is fetched. Across the webhdfs instances if
+ * same buffer block has been used then backend trip is avoided. Buffer block
+ * is certainly important since ADL fetches
+ * large amount of data (Default is 4MB however can be configured through
+ * core-site.xml) from the backend.
+ * Observation is in case of ORC/Avro kind of compressed file, buffer block
+ * does not avoid few backend calls across
+ * webhdfs
+ * instances.
+ */
+final class BufferManager {
+  private static final BufferManager BUFFER_MANAGER_INSTANCE = new
+      BufferManager();
+  private static Object lock = new Object();
+  private Buffer buffer = null;
+  private String fileName;
+
+  /**
+   * Constructor.
+   */
+  private BufferManager() {
+  }
+
+  public static Object getLock() {
+    return lock;
+  }
+
+  public static BufferManager getInstance() {
+    return BUFFER_MANAGER_INSTANCE;
+  }
+
+  /**
+   * Validate if the current buffer block is of given stream.
+   *
+   * @param path   ADL stream path
+   * @param offset Stream offset that caller is interested in
+   * @return True if the buffer block is available otherwise false
+   */
+  boolean hasValidDataForOffset(String path, long offset) {
+    if (this.fileName == null) {
+      return false;
+    }
+
+    if (!this.fileName.equals(path)) {
+      return false;
+    }
+
+    if (buffer == null) {
+      return false;
+    }
+
+    if ((offset < buffer.offset) || (offset >= (buffer.offset
+        + buffer.data.length))) {
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
+   * Clean buffer block.
+   */
+  void clear() {
+    buffer = null;
+  }
+
+  /**
+   * Validate if the current buffer block is of given stream. For now partial
+   * data available is not supported.
+   * Data must be available exactly or within the range of offset and size
+   * passed as parameter.
+   *
+   * @param path   Stream path
+   * @param offset Offset of the stream
+   * @param size   Size of the data from the offset of the stream caller
+   *               interested in
+   * @return True if the data is available from the given offset and of the
+   * size caller is interested in.
+   */
+  boolean hasData(String path, long offset, int size) {
+
+    if (!hasValidDataForOffset(path, offset)) {
+      return false;
+    }
+
+    if ((size + offset) > (buffer.data.length + buffer.offset)) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Return the buffer block from the requested offset. It is caller
+   * responsibility to check if the buffer block is
+   * of there interest and offset is valid.
+   *
+   * @param data   Byte array to be filed from the buffer block
+   * @param offset Data to be fetched from the offset.
+   */
+  void get(byte[] data, long offset) {
+    System.arraycopy(buffer.data, (int) (offset - buffer.offset), data, 0,
+        data.length);
+  }
+
+  /**
+   * Create new empty buffer block of the given size.
+   *
+   * @param len Size of the buffer block.
+   * @return Empty byte array.
+   */
+  byte[] getEmpty(int len) {
+    return new byte[len];
+  }
+
+  /**
+   * This function allows caller to specify new buffer block for the stream
+   * which is pulled from the backend.
+   *
+   * @param data   Buffer
+   * @param path   Stream path to which buffer belongs to
+   * @param offset Stream offset where buffer start with
+   */
+  void add(byte[] data, String path, long offset) {
+    if (data == null) {
+      return;
+    }
+
+    buffer = new Buffer();
+    buffer.data = data;
+    buffer.offset = offset;
+    this.fileName = path;
+  }
+
+  /**
+   * @return Size of the buffer.
+   */
+  int getBufferSize() {
+    return buffer.data.length;
+  }
+
+  /**
+   * @return Stream offset where buffer start with
+   */
+  long getBufferOffset() {
+    return buffer.offset;
+  }
+
+  /**
+   * Buffer container.
+   */
+  static class Buffer {
+    private byte[] data;
+    private long offset;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/PrivateAzureDataLakeFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/PrivateAzureDataLakeFileSystem.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/PrivateAzureDataLakeFileSystem.java
new file mode 100644
index 0000000..89011d2
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/PrivateAzureDataLakeFileSystem.java
@@ -0,0 +1,1108 @@
+/*
+ * 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.web;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.web.resources.ADLFlush;
+import org.apache.hadoop.hdfs.web.resources.ADLGetOpParam;
+import org.apache.hadoop.hdfs.web.resources.ADLPostOpParam;
+import org.apache.hadoop.hdfs.web.resources.ADLPutOpParam;
+import org.apache.hadoop.hdfs.web.resources.ADLVersionInfo;
+import org.apache.hadoop.hdfs.web.resources.AppendADLNoRedirectParam;
+import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
+import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
+import org.apache.hadoop.hdfs.web.resources.CreateADLNoRedirectParam;
+import org.apache.hadoop.hdfs.web.resources.CreateFlagParam;
+import org.apache.hadoop.hdfs.web.resources.CreateParentParam;
+import org.apache.hadoop.hdfs.web.resources.GetOpParam;
+import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
+import org.apache.hadoop.hdfs.web.resources.LeaseParam;
+import org.apache.hadoop.hdfs.web.resources.LengthParam;
+import org.apache.hadoop.hdfs.web.resources.OffsetParam;
+import org.apache.hadoop.hdfs.web.resources.OverwriteParam;
+import org.apache.hadoop.hdfs.web.resources.Param;
+import org.apache.hadoop.hdfs.web.resources.PermissionParam;
+import org.apache.hadoop.hdfs.web.resources.PutOpParam;
+import org.apache.hadoop.hdfs.web.resources.ReadADLNoRedirectParam;
+import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.VersionInfo;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.net.SocketException;
+import java.net.URI;
+import java.net.URL;
+import java.util.EnumSet;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Extended @see SWebHdfsFileSystem API. This class contains Azure data lake
+ * specific stability, Reliability and performance improvement.
+ * <p>
+ * Motivation behind PrivateAzureDataLakeFileSystem to encapsulate dependent
+ * implementation on org.apache.hadoop.hdfs.web package to configure query
+ * parameters, configuration over HTTP request send to backend .. etc. This
+ * class should be refactored and moved under package org.apache.hadoop.fs
+ * .adl once the required dependent changes are made into ASF code.
+ */
+public class PrivateAzureDataLakeFileSystem extends SWebHdfsFileSystem {
+
+  public static final String SCHEME = "adl";
+
+  // Feature configuration
+  private boolean featureGetBlockLocationLocallyBundled = true;
+  private boolean featureConcurrentReadWithReadAhead = true;
+  private boolean featureRedirectOff = true;
+  private boolean featureFlushWhenEOF = true;
+  private boolean overrideOwner = false;
+  private int maxConcurrentConnection;
+  private int maxBufferSize;
+  private String userName;
+
+  /**
+   * Constructor.
+   */
+  public PrivateAzureDataLakeFileSystem() {
+    try {
+      userName = UserGroupInformation.getCurrentUser().getShortUserName();
+    } catch (IOException e) {
+      userName = "hadoop";
+    }
+  }
+
+  @Override
+  public synchronized void initialize(URI uri, Configuration conf)
+      throws IOException {
+    super.initialize(uri, conf);
+    overrideOwner = getConf()
+        .getBoolean(ADLConfKeys.ADL_DEBUG_OVERRIDE_LOCAL_USER_AS_OWNER,
+            ADLConfKeys.ADL_DEBUG_SET_LOCAL_USER_AS_OWNER_DEFAULT);
+
+    featureRedirectOff = getConf()
+        .getBoolean(ADLConfKeys.ADL_FEATURE_REDIRECT_OFF,
+            ADLConfKeys.ADL_FEATURE_REDIRECT_OFF_DEFAULT);
+
+    featureGetBlockLocationLocallyBundled = getConf()
+        .getBoolean(ADLConfKeys.ADL_FEATURE_GET_BLOCK_LOCATION_LOCALLY_BUNDLED,
+            ADLConfKeys.ADL_FEATURE_GET_BLOCK_LOCATION_LOCALLY_BUNDLED_DEFAULT);
+
+    featureConcurrentReadWithReadAhead = getConf().
+        getBoolean(ADLConfKeys.ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD,
+            ADLConfKeys.ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD_DEFAULT);
+
+    maxBufferSize = getConf().getInt(
+        ADLConfKeys.ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD_BUFFER_SIZE,
+        ADLConfKeys
+            .ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD_BUFFER_SIZE_DEFAULT);
+
+    maxConcurrentConnection = getConf().getInt(
+        ADLConfKeys.ADL_FEATURE_CONCURRENT_READ_AHEAD_MAX_CONCURRENT_CONN,
+        ADLConfKeys
+            .ADL_FEATURE_CONCURRENT_READ_AHEAD_MAX_CONCURRENT_CONN_DEFAULT);
+  }
+
+  @VisibleForTesting
+  protected boolean isFeatureGetBlockLocationLocallyBundled() {
+    return featureGetBlockLocationLocallyBundled;
+  }
+
+  @VisibleForTesting
+  protected boolean isFeatureConcurrentReadWithReadAhead() {
+    return featureConcurrentReadWithReadAhead;
+  }
+
+  @VisibleForTesting
+  protected boolean isFeatureRedirectOff() {
+    return featureRedirectOff;
+  }
+
+  @VisibleForTesting
+  protected boolean isOverrideOwnerFeatureOn() {
+    return overrideOwner;
+  }
+
+  @VisibleForTesting
+  protected int getMaxBufferSize() {
+    return maxBufferSize;
+  }
+
+  @VisibleForTesting
+  protected int getMaxConcurrentConnection() {
+    return maxConcurrentConnection;
+  }
+
+  @Override
+  public String getScheme() {
+    return SCHEME;
+  }
+
+  /**
+   * Constructing home directory locally is fine as long as Hadoop
+   * local user name and ADL user name relationship story is not fully baked
+   * yet.
+   *
+   * @return Hadoop local user home directory.
+   */
+  @Override
+  public final Path getHomeDirectory() {
+    try {
+      return makeQualified(new Path(
+          "/user/" + UserGroupInformation.getCurrentUser().getShortUserName()));
+    } catch (IOException e) {
+    }
+
+    return new Path("/user/" + userName);
+  }
+
+  /**
+   * Azure data lake does not support user configuration for data replication
+   * hence not leaving system to query on
+   * azure data lake.
+   *
+   * Stub implementation
+   *
+   * @param p           Not honoured
+   * @param replication Not honoured
+   * @return True hard coded since ADL file system does not support
+   * replication configuration
+   * @throws IOException No exception would not thrown in this case however
+   *                     aligning with parent api definition.
+   */
+  @Override
+  public final boolean setReplication(final Path p, final short replication)
+      throws IOException {
+    return true;
+  }
+
+  /**
+   * @param f File/Folder path
+   * @return FileStatus instance containing metadata information of f
+   * @throws IOException For any system error
+   */
+  @Override
+  public FileStatus getFileStatus(Path f) throws IOException {
+    statistics.incrementReadOps(1);
+    FileStatus status = super.getFileStatus(f);
+
+    if (overrideOwner) {
+      FileStatus proxiedStatus = new FileStatus(status.getLen(),
+          status.isDirectory(), status.getReplication(), status.getBlockSize(),
+          status.getModificationTime(), status.getAccessTime(),
+          status.getPermission(), userName, "hdfs", status.getPath());
+      return proxiedStatus;
+    } else {
+      return status;
+    }
+  }
+
+  /**
+   * Create call semantic is handled differently in case of ADL. Create
+   * semantics is translated to Create/Append
+   * semantics.
+   * 1. No dedicated connection to server.
+   * 2. Buffering is locally done, Once buffer is full or flush is invoked on
+   * the by the caller. All the pending
+   * data is pushed to ADL as APPEND operation code.
+   * 3. On close - Additional call is send to server to close the stream, and
+   * release lock from the stream.
+   *
+   * Necessity of Create/Append semantics is
+   * 1. ADL backend server does not allow idle connection for longer duration
+   * . In case of slow writer scenario,
+   * observed connection timeout/Connection reset causing occasional job
+   * failures.
+   * 2. Performance boost to jobs which are slow writer, avoided network latency
+   * 3. ADL equally better performing with multiple of 4MB chunk as append
+   * calls.
+   *
+   * @param f           File path
+   * @param permission  Access permission for the newly created file
+   * @param overwrite   Remove existing file and recreate new one if true
+   *                    otherwise throw error if file exist
+   * @param bufferSize  Buffer size, ADL backend does not honour
+   * @param replication Replication count, ADL backend does not honour
+   * @param blockSize   Block size, ADL backend does not honour
+   * @param progress    Progress indicator
+   * @return FSDataOutputStream OutputStream on which application can push
+   * stream of bytes
+   * @throws IOException when system error, internal server error or user error
+   */
+  @Override
+  public FSDataOutputStream create(final Path f, final FsPermission permission,
+      final boolean overwrite, final int bufferSize, final short replication,
+      final long blockSize, final Progressable progress) throws IOException {
+    statistics.incrementWriteOps(1);
+
+    return new FSDataOutputStream(new BatchAppendOutputStream(f, bufferSize,
+        new PermissionParam(applyUMask(permission)),
+        new OverwriteParam(overwrite), new BufferSizeParam(bufferSize),
+        new ReplicationParam(replication), new BlockSizeParam(blockSize),
+        new ADLVersionInfo(VersionInfo.getVersion())), statistics) {
+    };
+  }
+
+  @Override
+  public FSDataOutputStream createNonRecursive(final Path f,
+      final FsPermission permission, final EnumSet<CreateFlag> flag,
+      final int bufferSize, final short replication, final long blockSize,
+      final Progressable progress) throws IOException {
+    statistics.incrementWriteOps(1);
+
+    String leaseId = java.util.UUID.randomUUID().toString();
+    return new FSDataOutputStream(new BatchAppendOutputStream(f, bufferSize,
+        new PermissionParam(applyUMask(permission)), new CreateFlagParam(flag),
+        new CreateParentParam(false), new BufferSizeParam(bufferSize),
+        new ReplicationParam(replication), new LeaseParam(leaseId),
+        new BlockSizeParam(blockSize),
+        new ADLVersionInfo(VersionInfo.getVersion())), statistics) {
+    };
+  }
+
+  /**
+   * Since defined as private in parent class, redefined to pass through
+   * Create api implementation.
+   *
+   * @param permission
+   * @return FsPermission list
+   */
+  private FsPermission applyUMask(FsPermission permission) {
+    FsPermission fsPermission = permission;
+    if (fsPermission == null) {
+      fsPermission = FsPermission.getDefault();
+    }
+    return fsPermission.applyUMask(FsPermission.getUMask(getConf()));
+  }
+
+  /**
+   * Open call semantic is handled differently in case of ADL. Instead of
+   * network stream is returned to the user,
+   * Overridden FsInputStream is returned.
+   *
+   * 1. No dedicated connection to server.
+   * 2. Process level concurrent read ahead Buffering is done, This allows
+   * data to be available for caller quickly.
+   * 3. Number of byte to read ahead is configurable.
+   *
+   * Advantage of Process level concurrent read ahead Buffering semantics is
+   * 1. ADL backend server does not allow idle connection for longer duration
+   * . In case of slow reader scenario,
+   * observed connection timeout/Connection reset causing occasional job
+   * failures.
+   * 2. Performance boost to jobs which are slow reader, avoided network latency
+   * 3. Compressed format support like ORC, and large data files gains the
+   * most out of this implementation.
+   *
+   * Read ahead feature is configurable.
+   *
+   * @param f          File path
+   * @param buffersize Buffer size
+   * @return FSDataInputStream InputStream on which application can read
+   * stream of bytes
+   * @throws IOException when system error, internal server error or user error
+   */
+  @Override
+  public FSDataInputStream open(final Path f, final int buffersize)
+      throws IOException {
+    statistics.incrementReadOps(1);
+
+    final HttpOpParam.Op op = GetOpParam.Op.OPEN;
+    // use a runner so the open can recover from an invalid token
+    FsPathConnectionRunner runner = null;
+
+    if (featureConcurrentReadWithReadAhead) {
+      URL url = this.toUrl(op, f, new BufferSizeParam(buffersize),
+          new ReadADLNoRedirectParam(true),
+          new ADLVersionInfo(VersionInfo.getVersion()));
+
+      BatchByteArrayInputStream bb = new BatchByteArrayInputStream(url, f,
+          maxBufferSize, maxConcurrentConnection);
+
+      FSDataInputStream fin = new FSDataInputStream(bb);
+      return fin;
+    } else {
+      if (featureRedirectOff) {
+        runner = new FsPathConnectionRunner(ADLGetOpParam.Op.OPEN, f,
+            new BufferSizeParam(buffersize), new ReadADLNoRedirectParam(true),
+            new ADLVersionInfo(VersionInfo.getVersion()));
+      } else {
+        runner = new FsPathConnectionRunner(op, f,
+            new BufferSizeParam(buffersize));
+      }
+
+      return new FSDataInputStream(
+          new OffsetUrlInputStream(new UnresolvedUrlOpener(runner),
+              new OffsetUrlOpener(null)));
+    }
+  }
+
+  /**
+   * @param f File/Folder path
+   * @return FileStatus array list
+   * @throws IOException For system error
+   */
+  @Override
+  public FileStatus[] listStatus(final Path f) throws IOException {
+    FileStatus[] fileStatuses = super.listStatus(f);
+    for (int i = 0; i < fileStatuses.length; i++) {
+      if (overrideOwner) {
+        fileStatuses[i] = new FileStatus(fileStatuses[i].getLen(),
+            fileStatuses[i].isDirectory(), fileStatuses[i].getReplication(),
+            fileStatuses[i].getBlockSize(),
+            fileStatuses[i].getModificationTime(),
+            fileStatuses[i].getAccessTime(), fileStatuses[i].getPermission(),
+            userName, "hdfs", fileStatuses[i].getPath());
+      }
+    }
+    return fileStatuses;
+  }
+
+  @Override
+  public BlockLocation[] getFileBlockLocations(final FileStatus status,
+      final long offset, final long length) throws IOException {
+    if (status == null) {
+      return null;
+    }
+
+    if (featureGetBlockLocationLocallyBundled) {
+      if ((offset < 0) || (length < 0)) {
+        throw new IllegalArgumentException("Invalid start or len parameter");
+      }
+
+      if (status.getLen() < offset) {
+        return new BlockLocation[0];
+      }
+
+      final String[] name = {"localhost"};
+      final String[] host = {"localhost"};
+      long blockSize = ADLConfKeys.DEFAULT_EXTENT_SIZE; // Block size must be
+      // non zero
+      int numberOfLocations =
+          (int) (length / blockSize) + ((length % blockSize == 0) ? 0 : 1);
+      BlockLocation[] locations = new BlockLocation[numberOfLocations];
+      for (int i = 0; i < locations.length; i++) {
+        long currentOffset = offset + (i * blockSize);
+        long currentLength = Math
+            .min(blockSize, offset + length - currentOffset);
+        locations[i] = new BlockLocation(name, host, currentOffset,
+            currentLength);
+      }
+
+      return locations;
+    } else {
+      return getFileBlockLocations(status.getPath(), offset, length);
+    }
+  }
+
+  @Override
+  public BlockLocation[] getFileBlockLocations(final Path p, final long offset,
+      final long length) throws IOException {
+    statistics.incrementReadOps(1);
+
+    if (featureGetBlockLocationLocallyBundled) {
+      FileStatus fileStatus = getFileStatus(p);
+      return getFileBlockLocations(fileStatus, offset, length);
+    } else {
+      return super.getFileBlockLocations(p, offset, length);
+    }
+  }
+
+  enum StreamState {
+    Initial,
+    DataCachedInLocalBuffer,
+    StreamEnd
+  }
+
+  class BatchAppendOutputStream extends OutputStream {
+    private Path fsPath;
+    private Param<?, ?>[] parameters;
+    private byte[] data = null;
+    private int offset = 0;
+    private long length = 0;
+    private boolean eof = false;
+    private boolean hadError = false;
+    private byte[] dataBuffers = null;
+    private int bufSize = 0;
+    private boolean streamClosed = false;
+
+    public BatchAppendOutputStream(Path path, int bufferSize,
+        Param<?, ?>... param) throws IOException {
+      if (bufferSize < (ADLConfKeys.DEFAULT_BLOCK_SIZE)) {
+        bufSize = ADLConfKeys.DEFAULT_BLOCK_SIZE;
+      } else {
+        bufSize = bufferSize;
+      }
+
+      this.fsPath = path;
+      this.parameters = param;
+      this.data = getBuffer();
+      FSDataOutputStream createStream = null;
+      try {
+        if (featureRedirectOff) {
+          CreateADLNoRedirectParam skipRedirect = new CreateADLNoRedirectParam(
+              true);
+          Param<?, ?>[] tmpParam = featureFlushWhenEOF ?
+              new Param<?, ?>[param.length + 2] :
+              new Param<?, ?>[param.length + 1];
+          System.arraycopy(param, 0, tmpParam, 0, param.length);
+          tmpParam[param.length] = skipRedirect;
+          if (featureFlushWhenEOF) {
+            tmpParam[param.length + 1] = new ADLFlush(false);
+          }
+          createStream = new FsPathOutputStreamRunner(ADLPutOpParam.Op.CREATE,
+              fsPath, 1, tmpParam).run();
+        } else {
+          createStream = new FsPathOutputStreamRunner(PutOpParam.Op.CREATE,
+              fsPath, 1, param).run();
+        }
+      } finally {
+        if (createStream != null) {
+          createStream.close();
+        }
+      }
+    }
+
+    @Override
+    public final synchronized void write(int b) throws IOException {
+      if (streamClosed) {
+        throw new IOException(fsPath + " stream object is closed.");
+      }
+
+      if (offset == (data.length)) {
+        flush();
+      }
+
+      data[offset] = (byte) b;
+      offset++;
+
+      // Statistics will get incremented again as part of the batch updates,
+      // decrement here to avoid double value
+      if (statistics != null) {
+        statistics.incrementBytesWritten(-1);
+      }
+    }
+
+    @Override
+    public final synchronized void write(byte[] buf, int off, int len)
+        throws IOException {
+      if (streamClosed) {
+        throw new IOException(fsPath + " stream object is closed.");
+      }
+
+      int bytesToWrite = len;
+      int localOff = off;
+      int localLen = len;
+      if (localLen >= data.length) {
+        // Flush data that is already in our internal buffer
+        flush();
+
+        // Keep committing data until we have less than our internal buffers
+        // length left
+        do {
+          try {
+            commit(buf, localOff, data.length, eof);
+          } catch (IOException e) {
+            hadError = true;
+            throw e;
+          }
+          localOff += data.length;
+          localLen -= data.length;
+        } while (localLen >= data.length);
+      }
+
+      // At this point, we have less than data.length left to copy from users
+      // buffer
+      if (offset + localLen >= data.length) {
+        // Users buffer has enough data left to fill our internal buffer
+        int bytesToCopy = data.length - offset;
+        System.arraycopy(buf, localOff, data, offset, bytesToCopy);
+        offset += bytesToCopy;
+
+        // Flush our internal buffer
+        flush();
+        localOff += bytesToCopy;
+        localLen -= bytesToCopy;
+      }
+
+      if (localLen > 0) {
+        // Simply copy the remainder from the users buffer into our internal
+        // buffer
+        System.arraycopy(buf, localOff, data, offset, localLen);
+        offset += localLen;
+      }
+
+      // Statistics will get incremented again as part of the batch updates,
+      // decrement here to avoid double value
+      if (statistics != null) {
+        statistics.incrementBytesWritten(-bytesToWrite);
+      }
+    }
+
+    @Override
+    public final synchronized void flush() throws IOException {
+      if (streamClosed) {
+        throw new IOException(fsPath + " stream object is closed.");
+      }
+
+      if (offset > 0) {
+        try {
+          commit(data, 0, offset, eof);
+        } catch (IOException e) {
+          hadError = true;
+          throw e;
+        }
+      }
+
+      offset = 0;
+    }
+
+    @Override
+    public final synchronized void close() throws IOException {
+      // Stream is closed earlier, return quietly.
+      if(streamClosed) {
+        return;
+      }
+
+      if (featureRedirectOff) {
+        eof = true;
+      }
+
+      boolean flushedSomething = false;
+      if (hadError) {
+        // No point proceeding further since the error has occurred and
+        // stream would be required to upload again.
+        streamClosed = true;
+        return;
+      } else {
+        flushedSomething = offset > 0;
+        try {
+          flush();
+        } finally {
+          streamClosed = true;
+        }
+      }
+
+      if (featureRedirectOff) {
+        // If we didn't flush anything from our internal buffer, we have to
+        // call the service again
+        // with an empty payload and flush=true in the url
+        if (!flushedSomething) {
+          try {
+            commit(null, 0, ADLConfKeys.KB, true);
+          } finally {
+            streamClosed = true;
+          }
+        }
+      }
+    }
+
+    private void commit(byte[] buffer, int off, int len, boolean endOfFile)
+        throws IOException {
+      OutputStream out = null;
+      try {
+        if (featureRedirectOff) {
+          AppendADLNoRedirectParam skipRedirect = new AppendADLNoRedirectParam(
+              true);
+          Param<?, ?>[] tmpParam = featureFlushWhenEOF ?
+              new Param<?, ?>[parameters.length + 3] :
+              new Param<?, ?>[parameters.length + 1];
+          System.arraycopy(parameters, 0, tmpParam, 0, parameters.length);
+          tmpParam[parameters.length] = skipRedirect;
+          if (featureFlushWhenEOF) {
+            tmpParam[parameters.length + 1] = new ADLFlush(endOfFile);
+            tmpParam[parameters.length + 2] = new OffsetParam(length);
+          }
+
+          out = new FsPathOutputStreamRunner(ADLPostOpParam.Op.APPEND, fsPath,
+              len, tmpParam).run();
+        } else {
+          out = new FsPathOutputStreamRunner(ADLPostOpParam.Op.APPEND, fsPath,
+              len, parameters).run();
+        }
+
+        if (buffer != null) {
+          out.write(buffer, off, len);
+          length += len;
+        }
+      } finally {
+        if (out != null) {
+          out.close();
+        }
+      }
+    }
+
+    private byte[] getBuffer() {
+      // Switch between the first and second buffer
+      dataBuffers = new byte[bufSize];
+      return dataBuffers;
+    }
+  }
+
+  /**
+   * Read data from backend in chunks instead of persistent connection. This
+   * is to avoid slow reader causing socket
+   * timeout.
+   */
+  protected class BatchByteArrayInputStream extends FSInputStream {
+
+    private static final int SIZE4MB = 4 * 1024 * 1024;
+    private final URL runner;
+    private byte[] data = null;
+    private long validDataHoldingSize = 0;
+    private int bufferOffset = 0;
+    private long currentFileOffset = 0;
+    private long nextFileOffset = 0;
+    private long fileSize = 0;
+    private StreamState state = StreamState.Initial;
+    private int maxBufferSize;
+    private int maxConcurrentConnection;
+    private Path fsPath;
+    private boolean streamIsClosed;
+    private Future[] subtasks = null;
+
+    BatchByteArrayInputStream(URL url, Path p, int bufferSize,
+        int concurrentConnection) throws IOException {
+      this.runner = url;
+      fsPath = p;
+      FileStatus fStatus = getFileStatus(fsPath);
+      if (!fStatus.isFile()) {
+        throw new IOException("Cannot open the directory " + p + " for " +
+            "reading");
+      }
+      fileSize = fStatus.getLen();
+      this.maxBufferSize = bufferSize;
+      this.maxConcurrentConnection = concurrentConnection;
+      this.streamIsClosed = false;
+    }
+
+    @Override
+    public synchronized final int read(long position, byte[] buffer, int offset,
+        int length) throws IOException {
+      if (streamIsClosed) {
+        throw new IOException("Stream already closed");
+      }
+      long oldPos = this.getPos();
+
+      int nread1;
+      try {
+        this.seek(position);
+        nread1 = this.read(buffer, offset, length);
+      } finally {
+        this.seek(oldPos);
+      }
+
+      return nread1;
+    }
+
+    @Override
+    public synchronized final int read() throws IOException {
+      if (streamIsClosed) {
+        throw new IOException("Stream already closed");
+      }
+      int status = doBufferAvailabilityCheck();
+      if (status == -1) {
+        return status;
+      }
+      int ch = data[bufferOffset++] & (0xff);
+      if (statistics != null) {
+        statistics.incrementBytesRead(1);
+      }
+      return ch;
+    }
+
+    @Override
+    public synchronized final void readFully(long position, byte[] buffer,
+        int offset, int length) throws IOException {
+      if (streamIsClosed) {
+        throw new IOException("Stream already closed");
+      }
+
+      super.readFully(position, buffer, offset, length);
+      if (statistics != null) {
+        statistics.incrementBytesRead(length);
+      }
+    }
+
+    @Override
+    public synchronized final int read(byte[] b, int off, int len)
+        throws IOException {
+      if (b == null) {
+        throw new IllegalArgumentException();
+      } else if (off < 0 || len < 0 || len > b.length - off) {
+        throw new IndexOutOfBoundsException();
+      } else if (len == 0) {
+        return 0;
+      }
+
+      if (streamIsClosed) {
+        throw new IOException("Stream already closed");
+      }
+      int status = doBufferAvailabilityCheck();
+      if (status == -1) {
+        return status;
+      }
+
+      int byteRead = 0;
+      long availableBytes = validDataHoldingSize - off;
+      long requestedBytes = bufferOffset + len - off;
+      if (requestedBytes <= availableBytes) {
+        System.arraycopy(data, bufferOffset, b, off, len);
+        bufferOffset += len;
+        byteRead = len;
+      } else {
+        byteRead = super.read(b, off, len);
+      }
+
+      if (statistics != null) {
+        statistics.incrementBytesRead(byteRead);
+      }
+
+      return byteRead;
+    }
+
+    private int doBufferAvailabilityCheck() throws IOException {
+      if (state == StreamState.Initial) {
+        validDataHoldingSize = fill(nextFileOffset);
+      }
+
+      long dataReloadSize = 0;
+      switch ((int) validDataHoldingSize) {
+      case -1:
+        state = StreamState.StreamEnd;
+        return -1;
+      case 0:
+        dataReloadSize = fill(nextFileOffset);
+        if (dataReloadSize <= 0) {
+          state = StreamState.StreamEnd;
+          return (int) dataReloadSize;
+        } else {
+          validDataHoldingSize = dataReloadSize;
+        }
+        break;
+      default:
+        break;
+      }
+
+      if (bufferOffset >= validDataHoldingSize) {
+        dataReloadSize = fill(nextFileOffset);
+      }
+
+      if (bufferOffset >= ((dataReloadSize == 0) ?
+          validDataHoldingSize :
+          dataReloadSize)) {
+        state = StreamState.StreamEnd;
+        return -1;
+      }
+
+      validDataHoldingSize = ((dataReloadSize == 0) ?
+          validDataHoldingSize :
+          dataReloadSize);
+      state = StreamState.DataCachedInLocalBuffer;
+      return 0;
+    }
+
+    private long fill(final long off) throws IOException {
+      if (state == StreamState.StreamEnd) {
+        return -1;
+      }
+
+      if (fileSize <= off) {
+        state = StreamState.StreamEnd;
+        return -1;
+      }
+      int len = maxBufferSize;
+      long fileOffset = 0;
+      boolean isEntireFileCached = true;
+      if ((fileSize <= maxBufferSize)) {
+        len = (int) fileSize;
+        currentFileOffset = 0;
+        nextFileOffset = 0;
+      } else {
+        if (len > (fileSize - off)) {
+          len = (int) (fileSize - off);
+        }
+
+        synchronized (BufferManager.getLock()) {
+          if (BufferManager.getInstance()
+              .hasValidDataForOffset(fsPath.toString(), off)) {
+            len = (int) (
+                BufferManager.getInstance().getBufferOffset() + BufferManager
+                    .getInstance().getBufferSize() - (int) off);
+          }
+        }
+
+        if (len <= 0) {
+          len = maxBufferSize;
+        }
+        fileOffset = off;
+        isEntireFileCached = false;
+      }
+
+      data = null;
+      BufferManager bm = BufferManager.getInstance();
+      data = bm.getEmpty(len);
+      boolean fetchDataOverNetwork = false;
+      synchronized (BufferManager.getLock()) {
+        if (bm.hasData(fsPath.toString(), fileOffset, len)) {
+          try {
+            bm.get(data, fileOffset);
+            validDataHoldingSize = data.length;
+            currentFileOffset = fileOffset;
+          } catch (ArrayIndexOutOfBoundsException e) {
+            fetchDataOverNetwork = true;
+          }
+        } else {
+          fetchDataOverNetwork = true;
+        }
+      }
+
+      if (fetchDataOverNetwork) {
+        int splitSize = getSplitSize(len);
+        try {
+          validDataHoldingSize = fillDataConcurrently(data, len, fileOffset,
+              splitSize);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new IOException("Interrupted filling buffer", e);
+        }
+
+        synchronized (BufferManager.getLock()) {
+          bm.add(data, fsPath.toString(), fileOffset);
+        }
+        currentFileOffset = nextFileOffset;
+      }
+
+      nextFileOffset += validDataHoldingSize;
+      state = StreamState.DataCachedInLocalBuffer;
+      bufferOffset = isEntireFileCached ? (int) off : 0;
+      return validDataHoldingSize;
+    }
+
+    int getSplitSize(int size) {
+      if (size <= SIZE4MB) {
+        return 1;
+      }
+
+      // Not practical
+      if (size > maxBufferSize) {
+        size = maxBufferSize;
+      }
+
+      int equalBufferSplit = Math.max(Math.round(size / SIZE4MB), 1);
+      int splitSize = Math.min(equalBufferSplit, maxConcurrentConnection);
+      return splitSize;
+    }
+
+    @Override
+    public synchronized final void seek(long pos) throws IOException {
+      if (pos == -1) {
+        throw new IOException("Bad offset, cannot seek to " + pos);
+      }
+
+      BufferManager bm = BufferManager.getInstance();
+      synchronized (BufferManager.getLock()) {
+        if (bm.hasValidDataForOffset(fsPath.toString(), pos)) {
+          state = StreamState.DataCachedInLocalBuffer;
+        } else if (pos >= 0) {
+          state = StreamState.Initial;
+        }
+      }
+
+      long availableBytes = (currentFileOffset + validDataHoldingSize);
+
+      // Check if this position falls under buffered data
+      if (pos < currentFileOffset || availableBytes <= 0) {
+        validDataHoldingSize = 0;
+        currentFileOffset = pos;
+        nextFileOffset = pos;
+        bufferOffset = 0;
+        return;
+      }
+
+      if (pos < availableBytes && pos >= currentFileOffset) {
+        state = StreamState.DataCachedInLocalBuffer;
+        bufferOffset = (int) (pos - currentFileOffset);
+      } else {
+        validDataHoldingSize = 0;
+        currentFileOffset = pos;
+        nextFileOffset = pos;
+        bufferOffset = 0;
+      }
+    }
+
+    @Override
+    public synchronized final long getPos() throws IOException {
+      if (streamIsClosed) {
+        throw new IOException("Stream already closed");
+      }
+      return bufferOffset + currentFileOffset;
+    }
+
+    @Override
+    public synchronized final int available() throws IOException {
+      if (streamIsClosed) {
+        throw new IOException("Stream already closed");
+      }
+      return Integer.MAX_VALUE;
+    }
+
+    @Override
+    public final boolean seekToNewSource(long targetPos) throws IOException {
+      return false;
+    }
+
+    @SuppressWarnings("unchecked")
+    private int fillDataConcurrently(byte[] byteArray, int length,
+        long globalOffset, int splitSize)
+        throws IOException, InterruptedException {
+      ExecutorService executor = Executors.newFixedThreadPool(splitSize);
+      subtasks = new Future[splitSize];
+      for (int i = 0; i < splitSize; i++) {
+        int offset = i * (length / splitSize);
+        int splitLength = (splitSize == (i + 1)) ?
+            (length / splitSize) + (length % splitSize) :
+            (length / splitSize);
+        subtasks[i] = executor.submit(
+            new BackgroundReadThread(byteArray, offset, splitLength,
+                globalOffset + offset));
+      }
+
+      executor.shutdown();
+      // wait until all tasks are finished
+      executor.awaitTermination(ADLConfKeys.DEFAULT_TIMEOUT_IN_SECONDS,
+          TimeUnit.SECONDS);
+
+      int totalBytePainted = 0;
+      for (int i = 0; i < splitSize; ++i) {
+        try {
+          totalBytePainted += (Integer) subtasks[i].get();
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new IOException(e.getCause());
+        } catch (ExecutionException e) {
+          Thread.currentThread().interrupt();
+          throw new IOException(e.getCause());
+        }
+      }
+
+      if (totalBytePainted != length) {
+        throw new IOException("Expected " + length + " bytes, Got " +
+            totalBytePainted + " bytes");
+      }
+
+      return totalBytePainted;
+    }
+
+    @Override
+    public synchronized final void close() throws IOException {
+      synchronized (BufferManager.getLock()) {
+        BufferManager.getInstance().clear();
+      }
+      //need to cleanup the above code the stream and connection close doesn't
+      // happen here
+      //flag set to mark close happened, cannot use the stream once closed
+      streamIsClosed = true;
+    }
+
+    /**
+     * Reads data from the ADL backend from the specified global offset and
+     * given
+     * length. Read data from ADL backend is copied to buffer array from the
+     * offset value specified.
+     *
+     * @param buffer       Store read data from ADL backend in the buffer.
+     * @param offset       Store read data from ADL backend in the buffer
+     *                     from the
+     *                     offset.
+     * @param length       Size of the data read from the ADL backend.
+     * @param globalOffset Read data from file offset.
+     * @return Number of bytes read from the ADL backend
+     * @throws IOException For any intermittent server issues or internal
+     *                     failures.
+     */
+    private int fillUpData(byte[] buffer, int offset, int length,
+        long globalOffset) throws IOException {
+      int totalBytesRead = 0;
+      final URL offsetUrl = new URL(
+          runner + "&" + new OffsetParam(String.valueOf(globalOffset)) + "&"
+              + new LengthParam(String.valueOf(length)));
+      HttpURLConnection conn = new URLRunner(GetOpParam.Op.OPEN, offsetUrl,
+          true).run();
+      InputStream in = conn.getInputStream();
+      try {
+        int bytesRead = 0;
+        while ((bytesRead = in.read(buffer, (int) offset + totalBytesRead,
+            (int) (length - totalBytesRead))) > 0) {
+          totalBytesRead += bytesRead;
+        }
+
+        // InputStream must be fully consumed to enable http keep-alive
+        if (bytesRead == 0) {
+          // Looking for EOF marker byte needs to be read.
+          if (in.read() != -1) {
+            throw new SocketException(
+                "Server returned more than requested data.");
+          }
+        }
+      } finally {
+        in.close();
+        conn.disconnect();
+      }
+
+      return totalBytesRead;
+    }
+
+    private class BackgroundReadThread implements Callable {
+
+      private final byte[] data;
+      private int offset;
+      private int length;
+      private long globalOffset;
+
+      BackgroundReadThread(byte[] buffer, int off, int size, long position) {
+        this.data = buffer;
+        this.offset = off;
+        this.length = size;
+        this.globalOffset = position;
+      }
+
+      public Object call() throws IOException {
+        return fillUpData(data, offset, length, globalOffset);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/PrivateCachedRefreshTokenBasedAccessTokenProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/PrivateCachedRefreshTokenBasedAccessTokenProvider.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/PrivateCachedRefreshTokenBasedAccessTokenProvider.java
new file mode 100644
index 0000000..d7dce25
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/PrivateCachedRefreshTokenBasedAccessTokenProvider.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.hdfs.web.oauth2;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * Exposing AccessTokenProvider publicly to extend in com.microsoft.azure
+ * .datalake package. Extended version to cache
+ * token for the process to gain performance gain.
+ */
+@Private
+@Unstable
+public abstract class PrivateCachedRefreshTokenBasedAccessTokenProvider
+    extends AccessTokenProvider {
+
+  // visibility workaround
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java
new file mode 100644
index 0000000..7a9dffa
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ *
+ */
+
+/**
+ * A distributed implementation of {@link
+ * org.apache.hadoop.hdfs.web.oauth2} for oauth2 token management support.
+ */
+package org.apache.hadoop.hdfs.web.oauth2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/package-info.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/package-info.java
new file mode 100644
index 0000000..1cc8273
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/package-info.java
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ *
+ */
+
+/**
+ * A distributed implementation of {@link org.apache.hadoop.hdfs.web} for
+ * reading and writing files on Azure data lake file system. This
+ * implementation is derivation from the webhdfs specification.
+ */
+package org.apache.hadoop.hdfs.web;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLFlush.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLFlush.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLFlush.java
new file mode 100644
index 0000000..b76aaaa
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLFlush.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.hdfs.web.resources;
+
+/**
+ * Query parameter to notify backend server that the all the data has been
+ * pushed to over the stream.
+ *
+ * Used in operation code Create and Append.
+ */
+public class ADLFlush extends BooleanParam {
+  /**
+   * Parameter name.
+   */
+  public static final String NAME = "flush";
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   *
+   * @param value the parameter value.
+   */
+  public ADLFlush(final Boolean value) {
+    super(DOMAIN, value);
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9581fb71/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLGetOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLGetOpParam.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLGetOpParam.java
new file mode 100644
index 0000000..6b3708f
--- /dev/null
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLGetOpParam.java
@@ -0,0 +1,96 @@
+/*
+ * 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.web.resources;
+
+import java.net.HttpURLConnection;
+
+/**
+ * Extended Webhdfs GetOpParam to avoid redirect operation for azure data
+ * lake storage.
+ */
+public class ADLGetOpParam extends HttpOpParam<ADLGetOpParam.Op> {
+  private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
+
+  /**
+   * Constructor.
+   *
+   * @param str a string representation of the parameter value.
+   */
+  public ADLGetOpParam(final String str) {
+    super(DOMAIN, DOMAIN.parse(str));
+  }
+
+  @Override
+  public final String getName() {
+    return NAME;
+  }
+
+  /**
+   * Get operations.
+   */
+  public static enum Op implements HttpOpParam.Op {
+    OPEN(false, HttpURLConnection.HTTP_OK);
+
+    private final boolean redirect;
+    private final int expectedHttpResponseCode;
+    private final boolean requireAuth;
+
+    Op(final boolean doRedirect, final int expectHttpResponseCode) {
+      this(doRedirect, expectHttpResponseCode, false);
+    }
+
+    Op(final boolean doRedirect, final int expectHttpResponseCode,
+        final boolean doRequireAuth) {
+      this.redirect = doRedirect;
+      this.expectedHttpResponseCode = expectHttpResponseCode;
+      this.requireAuth = doRequireAuth;
+    }
+
+    @Override
+    public HttpOpParam.Type getType() {
+      return HttpOpParam.Type.GET;
+    }
+
+    @Override
+    public boolean getRequireAuth() {
+      return requireAuth;
+    }
+
+    @Override
+    public boolean getDoOutput() {
+      return false;
+    }
+
+    @Override
+    public boolean getRedirect() {
+      return redirect;
+    }
+
+    @Override
+    public int getExpectedHttpResponseCode() {
+      return expectedHttpResponseCode;
+    }
+
+    @Override
+    public String toQueryString() {
+      return NAME + "=" + this;
+    }
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[12/51] [abbrv] hadoop git commit: HDFS-10220. A large number of expired leases can make namenode unresponsive and cause failover (Nicolas Fraison via raviprak)

Posted by vv...@apache.org.
HDFS-10220. A large number of expired leases can make namenode unresponsive and cause failover (Nicolas Fraison via raviprak)


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

Branch: refs/heads/YARN-3926
Commit: ae047655f4355288406cd5396fb4e3ea7c307b14
Parents: 0af96a1
Author: Ravi Prakash <ra...@altiscale.com>
Authored: Wed Jun 8 13:44:22 2016 -0700
Committer: Ravi Prakash <ra...@altiscale.com>
Committed: Wed Jun 8 13:44:22 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   | 10 +++++
 .../hdfs/server/common/HdfsServerConstants.java |  1 -
 .../hdfs/server/namenode/FSNamesystem.java      | 42 ++++++++++++++++----
 .../hdfs/server/namenode/LeaseManager.java      | 21 ++++++++--
 .../src/main/resources/hdfs-default.xml         | 18 +++++++++
 .../hdfs/server/namenode/TestLeaseManager.java  | 24 ++++++-----
 6 files changed, 94 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae047655/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 19e1791..f18a6c6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -397,6 +397,16 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_NAMENODE_MAX_XATTR_SIZE_DEFAULT = 16384;
   public static final int     DFS_NAMENODE_MAX_XATTR_SIZE_HARD_LIMIT = 32768;
 
+  public static final String  DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_KEY =
+      "dfs.namenode.lease-recheck-interval-ms";
+  public static final long    DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_DEFAULT =
+      2000;
+  public static final String
+      DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_KEY =
+      "dfs.namenode.max-lock-hold-to-release-lease-ms";
+  public static final long
+      DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_DEFAULT = 25;
+
   public static final String  DFS_UPGRADE_DOMAIN_FACTOR = "dfs.namenode.upgrade.domain.factor";
   public static final int DFS_UPGRADE_DOMAIN_FACTOR_DEFAULT = DFS_REPLICATION_DEFAULT;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae047655/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
index b2dda3c..3798394 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
@@ -361,7 +361,6 @@ public interface HdfsServerConstants {
   }
   
   String NAMENODE_LEASE_HOLDER = "HDFS_NameNode";
-  long NAMENODE_LEASE_RECHECK_INTERVAL = 2000;
 
   String CRYPTO_XATTR_ENCRYPTION_ZONE =
       "raw.hdfs.crypto.encryption.zone";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae047655/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 c9f2487..915ae97 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
@@ -76,6 +76,10 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_EXPI
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT;
@@ -385,7 +389,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   private final UserGroupInformation fsOwner;
   private final String supergroup;
   private final boolean standbyShouldCheckpoint;
-  
+
+  /** Interval between each check of lease to release. */
+  private final long leaseRecheckIntervalMs;
+  /** Maximum time the lock is hold to release lease. */
+  private final long maxLockHoldToReleaseLeaseMs;
+
   // Scan interval is not configurable.
   private static final long DELEGATION_TOKEN_REMOVER_SCAN_INTERVAL =
     TimeUnit.MILLISECONDS.convert(1, TimeUnit.HOURS);
@@ -803,6 +812,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           DFSConfigKeys.DFS_NAMENODE_EDEKCACHELOADER_INTERVAL_MS_KEY,
           DFSConfigKeys.DFS_NAMENODE_EDEKCACHELOADER_INTERVAL_MS_DEFAULT);
 
+      this.leaseRecheckIntervalMs = conf.getLong(
+          DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_KEY,
+          DFS_NAMENODE_LEASE_RECHECK_INTERVAL_MS_DEFAULT);
+      this.maxLockHoldToReleaseLeaseMs = conf.getLong(
+          DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_KEY,
+          DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_DEFAULT);
+
       // For testing purposes, allow the DT secret manager to be started regardless
       // of whether security is enabled.
       alwaysUseDelegationTokensForTests = conf.getBoolean(
@@ -847,6 +863,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return retryCache;
   }
 
+  @VisibleForTesting
+  public long getLeaseRecheckIntervalMs() {
+    return leaseRecheckIntervalMs;
+  }
+
+  @VisibleForTesting
+  public long getMaxLockHoldToReleaseLeaseMs() {
+    return maxLockHoldToReleaseLeaseMs;
+  }
+
   void lockRetryCache() {
     if (retryCache != null) {
       retryCache.lock();
@@ -3116,9 +3142,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     if(nrCompleteBlocks == nrBlocks) {
       finalizeINodeFileUnderConstruction(src, pendingFile,
           iip.getLatestSnapshotId(), false);
-      NameNode.stateChangeLog.warn("BLOCK*"
-        + " internalReleaseLease: All existing blocks are COMPLETE,"
-        + " lease removed, file closed.");
+      NameNode.stateChangeLog.warn("BLOCK*" +
+          " internalReleaseLease: All existing blocks are COMPLETE," +
+          " lease removed, file " + src + " closed.");
       return true;  // closed!
     }
 
@@ -3155,9 +3181,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           blockManager.hasMinStorage(lastBlock)) {
         finalizeINodeFileUnderConstruction(src, pendingFile,
             iip.getLatestSnapshotId(), false);
-        NameNode.stateChangeLog.warn("BLOCK*"
-          + " internalReleaseLease: Committed blocks are minimally replicated,"
-          + " lease removed, file closed.");
+        NameNode.stateChangeLog.warn("BLOCK*" +
+            " internalReleaseLease: Committed blocks are minimally" +
+            " replicated, lease removed, file" + src + " closed.");
         return true;  // closed!
       }
       // Cannot close file right now, since some blocks 
@@ -3200,7 +3226,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         finalizeINodeFileUnderConstruction(src, pendingFile,
             iip.getLatestSnapshotId(), false);
         NameNode.stateChangeLog.warn("BLOCK* internalReleaseLease: "
-            + "Removed empty last block and closed file.");
+            + "Removed empty last block and closed file " + src);
         return true;
       }
       // start recovery of the last block for this file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae047655/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
index e97aa53..06f6586 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
@@ -336,7 +336,7 @@ public class LeaseManager {
             }
           }
   
-          Thread.sleep(HdfsServerConstants.NAMENODE_LEASE_RECHECK_INTERVAL);
+          Thread.sleep(fsnamesystem.getLeaseRecheckIntervalMs());
         } catch(InterruptedException ie) {
           if (LOG.isDebugEnabled()) {
             LOG.debug(name + " is interrupted", ie);
@@ -356,8 +356,11 @@ public class LeaseManager {
     boolean needSync = false;
     assert fsnamesystem.hasWriteLock();
 
-    while(!sortedLeases.isEmpty() && sortedLeases.peek().expiredHardLimit()) {
-      Lease leaseToCheck = sortedLeases.poll();
+    long start = monotonicNow();
+
+    while(!sortedLeases.isEmpty() && sortedLeases.peek().expiredHardLimit()
+      && !isMaxLockHoldToReleaseLease(start)) {
+      Lease leaseToCheck = sortedLeases.peek();
       LOG.info(leaseToCheck + " has expired hard limit");
 
       final List<Long> removing = new ArrayList<>();
@@ -397,6 +400,11 @@ public class LeaseManager {
               + leaseToCheck, e);
           removing.add(id);
         }
+        if (isMaxLockHoldToReleaseLease(start)) {
+          LOG.debug("Breaking out of checkLeases after " +
+              fsnamesystem.getMaxLockHoldToReleaseLeaseMs() + "ms.");
+          break;
+        }
       }
 
       for(Long id : removing) {
@@ -407,6 +415,13 @@ public class LeaseManager {
     return needSync;
   }
 
+
+  /** @return true if max lock hold is reached */
+  private boolean isMaxLockHoldToReleaseLease(long start) {
+    return monotonicNow() - start >
+        fsnamesystem.getMaxLockHoldToReleaseLeaseMs();
+  }
+
   @Override
   public synchronized String toString() {
     return getClass().getSimpleName() + "= {"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae047655/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 79f7911..fc2f942 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -2590,6 +2590,24 @@
 </property>
 
 <property>
+  <name>dfs.namenode.lease-recheck-interval-ms</name>
+  <value>2000</value>
+  <description>During the release of lease a lock is hold that make any
+    operations on the namenode stuck. In order to not block them during
+    a too long duration we stop releasing lease after this max lock limit.
+  </description>
+</property>
+
+<property>
+  <name>dfs.namenode.max-lock-hold-to-release-lease-ms</name>
+  <value>25</value>
+  <description>During the release of lease a lock is hold that make any
+    operations on the namenode stuck. In order to not block them during
+    a too long duration we stop releasing lease after this max lock limit.
+  </description>
+</property>
+
+<property>
   <name>dfs.namenode.startup.delay.block.deletion.sec</name>
   <value>0</value>
   <description>The delay in seconds at which we will pause the blocks deletion

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae047655/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
index 3bb7bb7..f823745 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 
@@ -39,6 +40,8 @@ public class TestLeaseManager {
   @Rule
   public Timeout timeout = new Timeout(300000);
 
+  public static long maxLockHoldToReleaseLeaseMs = 100;
+
   @Test
   public void testRemoveLeases() throws Exception {
     FSNamesystem fsn = mock(FSNamesystem.class);
@@ -57,28 +60,28 @@ public class TestLeaseManager {
     assertEquals(0, lm.getINodeIdWithLeases().size());
   }
 
-  /** Check that even if LeaseManager.checkLease is not able to relinquish
-   * leases, the Namenode does't enter an infinite loop while holding the FSN
-   * write lock and thus become unresponsive
+  /** Check that LeaseManager.checkLease release some leases
    */
   @Test
-  public void testCheckLeaseNotInfiniteLoop() {
+  public void testCheckLease() {
     LeaseManager lm = new LeaseManager(makeMockFsNameSystem());
 
+    long numLease = 100;
+
     //Make sure the leases we are going to add exceed the hard limit
     lm.setLeasePeriod(0, 0);
 
-    //Add some leases to the LeaseManager
-    lm.addLease("holder1", INodeId.ROOT_INODE_ID + 1);
-    lm.addLease("holder2", INodeId.ROOT_INODE_ID + 2);
-    lm.addLease("holder3", INodeId.ROOT_INODE_ID + 3);
-    assertEquals(lm.countLease(), 3);
+    for (long i = 0; i <= numLease - 1; i++) {
+      //Add some leases to the LeaseManager
+      lm.addLease("holder"+i, INodeId.ROOT_INODE_ID + i);
+    }
+    assertEquals(numLease, lm.countLease());
 
     //Initiate a call to checkLease. This should exit within the test timeout
     lm.checkLeases();
+    assertTrue(lm.countLease() < numLease);
   }
 
-
   @Test
   public void testCountPath() {
     LeaseManager lm = new LeaseManager(makeMockFsNameSystem());
@@ -112,6 +115,7 @@ public class TestLeaseManager {
     when(fsn.isRunning()).thenReturn(true);
     when(fsn.hasWriteLock()).thenReturn(true);
     when(fsn.getFSDirectory()).thenReturn(dir);
+    when(fsn.getMaxLockHoldToReleaseLeaseMs()).thenReturn(maxLockHoldToReleaseLeaseMs);
     return fsn;
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org