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 zh...@apache.org on 2015/01/05 23:49:53 UTC

[01/18] hadoop git commit: YARN-2493. Added node-labels page on RM web UI. Contributed by Wangda Tan

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-EC 43dd72d9b -> e38cd055b


YARN-2493. Added node-labels page on RM web UI. Contributed by Wangda Tan


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

Branch: refs/heads/HDFS-EC
Commit: d5a33b4acb03d7e27baf7e4cda91a0a1563941dd
Parents: 181827a
Author: Jian He <ji...@apache.org>
Authored: Tue Dec 30 15:35:45 2014 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:36 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   2 +
 .../nodelabels/CommonNodeLabelsManager.java     |  21 +--
 .../hadoop/yarn/nodelabels/NodeLabel.java       |  96 +++++++++++++
 .../hadoop/yarn/webapp/YarnWebParams.java       |   1 +
 .../nodelabels/RMNodeLabelsManager.java         |  36 +++--
 .../server/resourcemanager/webapp/NavBlock.java |   3 +-
 .../resourcemanager/webapp/NodeLabelsPage.java  |  91 ++++++++++++
 .../resourcemanager/webapp/NodesPage.java       | 139 ++++++++++---------
 .../server/resourcemanager/webapp/RMWebApp.java |   1 +
 .../resourcemanager/webapp/RmController.java    |   6 +-
 .../yarn/server/resourcemanager/MockNodes.java  |  46 +++++-
 .../TestWorkPreservingRMRestart.java            |   2 +-
 .../nodelabels/TestRMNodeLabelsManager.java     |  33 +++++
 .../resourcemanager/webapp/TestNodesPage.java   |  45 ++++++
 14 files changed, 421 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a33b4a/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e707ea7..0ebf8a3 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -155,6 +155,8 @@ Release 2.7.0 - UNRELEASED
     YARN-2993. Several fixes (missing acl check, error log msg ...) and some 
     refinement in AdminService. (Yi Liu via junping_du)
 
+    YARN-2943. Added node-labels page on RM web UI. (Wangda Tan via jianhe)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a33b4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
index 070aa1f..e888cc5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
@@ -72,8 +72,8 @@ public class CommonNodeLabelsManager extends AbstractService {
 
   protected Dispatcher dispatcher;
 
-  protected ConcurrentMap<String, Label> labelCollections =
-      new ConcurrentHashMap<String, Label>();
+  protected ConcurrentMap<String, NodeLabel> labelCollections =
+      new ConcurrentHashMap<String, NodeLabel>();
   protected ConcurrentMap<String, Host> nodeCollections =
       new ConcurrentHashMap<String, Host>();
 
@@ -82,19 +82,6 @@ public class CommonNodeLabelsManager extends AbstractService {
 
   protected NodeLabelsStore store;
 
-  protected static class Label {
-    private Resource resource;
-
-    protected Label() {
-      this.resource = Resource.newInstance(0, 0);
-    }
-
-    public Resource getResource() {
-      return this.resource;
-    }
-
-  }
-
   /**
    * A <code>Host</code> can have multiple <code>Node</code>s 
    */
@@ -201,7 +188,7 @@ public class CommonNodeLabelsManager extends AbstractService {
   protected void serviceInit(Configuration conf) throws Exception {
     initNodeLabelStore(conf);
     
-    labelCollections.put(NO_LABEL, new Label());
+    labelCollections.put(NO_LABEL, new NodeLabel(NO_LABEL));
   }
 
   protected void initNodeLabelStore(Configuration conf) throws Exception {
@@ -271,7 +258,7 @@ public class CommonNodeLabelsManager extends AbstractService {
     for (String label : labels) {
       // shouldn't overwrite it to avoid changing the Label.resource
       if (this.labelCollections.get(label) == null) {
-        this.labelCollections.put(label, new Label());
+        this.labelCollections.put(label, new NodeLabel(label));
         newLabels.add(label);
       }
     }

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a33b4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
index 91d2a20..62c3c7a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
@@ -32,4 +32,5 @@ public interface YarnWebParams {
   String APP_STATE = "app.state";
   String QUEUE_NAME = "queue.name";
   String NODE_STATE = "node.state";
+  String NODE_LABEL = "node.label";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a33b4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
index 646441a..828d1bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
@@ -19,10 +19,12 @@
 package org.apache.hadoop.yarn.server.resourcemanager.nodelabels;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -37,6 +39,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.nodelabels.NodeLabel;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeLabelsUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -360,8 +363,8 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         // no label in the past
         if (oldLabels.isEmpty()) {
           // update labels
-          Label label = labelCollections.get(NO_LABEL);
-          Resources.subtractFrom(label.getResource(), oldNM.resource);
+          NodeLabel label = labelCollections.get(NO_LABEL);
+          label.removeNode(oldNM.resource);
 
           // update queues, all queue can access this node
           for (Queue q : queueCollections.values()) {
@@ -370,11 +373,11 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         } else {
           // update labels
           for (String labelName : oldLabels) {
-            Label label = labelCollections.get(labelName);
+            NodeLabel label = labelCollections.get(labelName);
             if (null == label) {
               continue;
             }
-            Resources.subtractFrom(label.getResource(), oldNM.resource);
+            label.removeNode(oldNM.resource);
           }
 
           // update queues, only queue can access this node will be subtract
@@ -395,8 +398,8 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         // no label in the past
         if (newLabels.isEmpty()) {
           // update labels
-          Label label = labelCollections.get(NO_LABEL);
-          Resources.addTo(label.getResource(), newNM.resource);
+          NodeLabel label = labelCollections.get(NO_LABEL);
+          label.addNode(newNM.resource);
 
           // update queues, all queue can access this node
           for (Queue q : queueCollections.values()) {
@@ -405,8 +408,8 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         } else {
           // update labels
           for (String labelName : newLabels) {
-            Label label = labelCollections.get(labelName);
-            Resources.addTo(label.getResource(), newNM.resource);
+            NodeLabel label = labelCollections.get(labelName);
+            label.addNode(newNM.resource);
           }
 
           // update queues, only queue can access this node will be subtract
@@ -475,4 +478,21 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
   public void setRMContext(RMContext rmContext) {
     this.rmContext = rmContext;
   }
+
+  public List<NodeLabel> pullRMNodeLabelsInfo() {
+    try {
+      readLock.lock();
+      List<NodeLabel> infos = new ArrayList<NodeLabel>();
+
+      for (Entry<String, NodeLabel> entry : labelCollections.entrySet()) {
+        NodeLabel label = entry.getValue();
+        infos.add(label.getCopy());
+      }
+
+      Collections.sort(infos);
+      return infos;
+    } finally {
+      readLock.unlock();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a33b4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.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/NavBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.java
index ce8fd9e..db00bb0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.java
@@ -33,7 +33,8 @@ public class NavBlock extends HtmlBlock {
         h3("Cluster").
         ul().
           li().a(url("cluster"), "About")._().
-          li().a(url("nodes"), "Nodes")._();
+          li().a(url("nodes"), "Nodes")._().
+          li().a(url("nodelabels"), "Node Labels")._();
     UL<LI<UL<DIV<Hamlet>>>> subAppsList = mainList.
           li().a(url("apps"), "Applications").
             ul();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a33b4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java
new file mode 100644
index 0000000..5e8c1ed
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java
@@ -0,0 +1,91 @@
+/**
+ * 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;
+
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID;
+
+import org.apache.hadoop.yarn.nodelabels.NodeLabel;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.webapp.SubView;
+import org.apache.hadoop.yarn.webapp.YarnWebParams;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+
+import com.google.inject.Inject;
+
+public class NodeLabelsPage extends RmView {
+  static class NodeLabelsBlock extends HtmlBlock {
+    final ResourceManager rm;
+
+    @Inject
+    NodeLabelsBlock(ResourceManager rm, ViewContext ctx) {
+      super(ctx);
+      this.rm = rm;
+    }
+
+    @Override
+    protected void render(Block html) {
+      TBODY<TABLE<Hamlet>> tbody = html.table("#nodelabels").
+          thead().
+          tr().
+          th(".name", "Label Name").
+          th(".numOfActiveNMs", "Num Of Active NMs").
+          th(".totalResource", "Total Resource").
+          _()._().
+          tbody();
+  
+      RMNodeLabelsManager nlm = rm.getRMContext().getNodeLabelManager();
+      for (NodeLabel info : nlm.pullRMNodeLabelsInfo()) {
+        TR<TBODY<TABLE<Hamlet>>> row =
+            tbody.tr().td(
+                info.getLabelName().isEmpty() ? "<NO_LABEL>" : info
+                    .getLabelName());
+        int nActiveNMs = info.getNumActiveNMs();
+        if (nActiveNMs > 0) {
+          row = row.td()
+          .a(url("nodes",
+              "?" + YarnWebParams.NODE_LABEL + "=" + info.getLabelName()),
+              String.valueOf(nActiveNMs))
+           ._();
+        } else {
+          row = row.td(String.valueOf(nActiveNMs));
+        }
+        row.td(info.getResource().toString())._();
+      }
+      tbody._()._();
+    }
+  }
+
+  @Override protected void preHead(Page.HTML<_> html) {
+    commonPreHead(html);
+    String title = "Node labels of the cluster";
+    setTitle(title);
+    set(DATATABLES_ID, "nodelabels");
+    setTableStyles(html, "nodelabels", ".healthStatus {width:10em}",
+                   ".healthReport {width:10em}");
+  }
+
+  @Override protected Class<? extends SubView> content() {
+    return NodeLabelsBlock.class;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a33b4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.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/NodesPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java
index d3849ae..f28a9a8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java
@@ -1,24 +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.
-*/
+ * 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;
 
-import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebApp.NODE_STATE;
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.NODE_STATE;
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.NODE_LABEL;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
@@ -28,7 +29,9 @@ import java.util.Collection;
 
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
@@ -60,26 +63,20 @@ class NodesPage extends RmView {
 
       ResourceScheduler sched = rm.getResourceScheduler();
       String type = $(NODE_STATE);
-      TBODY<TABLE<Hamlet>> tbody = html.table("#nodes").
-          thead().
-          tr().
-          th(".nodelabels", "Node Labels").
-          th(".rack", "Rack").
-          th(".state", "Node State").
-          th(".nodeaddress", "Node Address").
-          th(".nodehttpaddress", "Node HTTP Address").
-          th(".lastHealthUpdate", "Last health-update").
-          th(".healthReport", "Health-report").
-          th(".containers", "Containers").
-          th(".mem", "Mem Used").
-          th(".mem", "Mem Avail").
-          th(".vcores", "VCores Used").
-          th(".vcores", "VCores Avail").
-          th(".nodeManagerVersion", "Version").
-          _()._().
-          tbody();
+      String labelFilter = $(NODE_LABEL, CommonNodeLabelsManager.ANY).trim();
+      TBODY<TABLE<Hamlet>> tbody =
+          html.table("#nodes").thead().tr().th(".nodelabels", "Node Labels")
+              .th(".rack", "Rack").th(".state", "Node State")
+              .th(".nodeaddress", "Node Address")
+              .th(".nodehttpaddress", "Node HTTP Address")
+              .th(".lastHealthUpdate", "Last health-update")
+              .th(".healthReport", "Health-report")
+              .th(".containers", "Containers").th(".mem", "Mem Used")
+              .th(".mem", "Mem Avail").th(".vcores", "VCores Used")
+              .th(".vcores", "VCores Avail")
+              .th(".nodeManagerVersion", "Version")._()._().tbody();
       NodeState stateFilter = null;
-      if(type != null && !type.isEmpty()) {
+      if (type != null && !type.isEmpty()) {
         stateFilter = NodeState.valueOf(type.toUpperCase());
       }
       Collection<RMNode> rmNodes = this.rm.getRMContext().getRMNodes().values();
@@ -97,9 +94,9 @@ class NodesPage extends RmView {
         }
       }
       for (RMNode ni : rmNodes) {
-        if(stateFilter != null) {
+        if (stateFilter != null) {
           NodeState state = ni.getState();
-          if(!stateFilter.equals(state)) {
+          if (!stateFilter.equals(state)) {
             continue;
           }
         } else {
@@ -109,61 +106,71 @@ class NodesPage extends RmView {
             continue;
           }
         }
+        // Besides state, we need to filter label as well.
+        if (!labelFilter.equals(RMNodeLabelsManager.ANY)) {
+          if (labelFilter.isEmpty()) {
+            // Empty label filter means only shows nodes without label
+            if (!ni.getNodeLabels().isEmpty()) {
+              continue;
+            }
+          } else if (!ni.getNodeLabels().contains(labelFilter)) {
+            // Only nodes have given label can show on web page.
+            continue;
+          }
+        }
         NodeInfo info = new NodeInfo(ni, sched);
-        int usedMemory = (int)info.getUsedMemory();
-        int availableMemory = (int)info.getAvailableMemory();
-        TR<TBODY<TABLE<Hamlet>>> row = tbody.tr().
-            td(StringUtils.join(",", info.getNodeLabels())).
-            td(info.getRack()).
-            td(info.getState()).
-            td(info.getNodeId());
+        int usedMemory = (int) info.getUsedMemory();
+        int availableMemory = (int) info.getAvailableMemory();
+        TR<TBODY<TABLE<Hamlet>>> row =
+            tbody.tr().td(StringUtils.join(",", info.getNodeLabels()))
+                .td(info.getRack()).td(info.getState()).td(info.getNodeId());
         if (isInactive) {
           row.td()._("N/A")._();
         } else {
           String httpAddress = info.getNodeHTTPAddress();
-          row.td().a("//" + httpAddress,
-              httpAddress)._();
+          row.td().a("//" + httpAddress, httpAddress)._();
         }
-        row.td().br().$title(String.valueOf(info.getLastHealthUpdate()))._().
-              _(Times.format(info.getLastHealthUpdate()))._().
-            td(info.getHealthReport()).
-            td(String.valueOf(info.getNumContainers())).
-            td().br().$title(String.valueOf(usedMemory))._().
-              _(StringUtils.byteDesc(usedMemory * BYTES_IN_MB))._().
-            td().br().$title(String.valueOf(availableMemory))._().
-              _(StringUtils.byteDesc(availableMemory * BYTES_IN_MB))._().
-              td(String.valueOf(info.getUsedVirtualCores())).
-              td(String.valueOf(info.getAvailableVirtualCores())).
-            td(ni.getNodeManagerVersion()).
-            _();
+        row.td().br().$title(String.valueOf(info.getLastHealthUpdate()))._()
+            ._(Times.format(info.getLastHealthUpdate()))._()
+            .td(info.getHealthReport())
+            .td(String.valueOf(info.getNumContainers())).td().br()
+            .$title(String.valueOf(usedMemory))._()
+            ._(StringUtils.byteDesc(usedMemory * BYTES_IN_MB))._().td().br()
+            .$title(String.valueOf(availableMemory))._()
+            ._(StringUtils.byteDesc(availableMemory * BYTES_IN_MB))._()
+            .td(String.valueOf(info.getUsedVirtualCores()))
+            .td(String.valueOf(info.getAvailableVirtualCores()))
+            .td(ni.getNodeManagerVersion())._();
       }
       tbody._()._();
     }
   }
 
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override
+  protected void preHead(Page.HTML<_> html) {
     commonPreHead(html);
     String type = $(NODE_STATE);
     String title = "Nodes of the cluster";
-    if(type != null && !type.isEmpty()) {
-      title = title+" ("+type+")";
+    if (type != null && !type.isEmpty()) {
+      title = title + " (" + type + ")";
     }
     setTitle(title);
     set(DATATABLES_ID, "nodes");
     set(initID(DATATABLES, "nodes"), nodesTableInit());
     setTableStyles(html, "nodes", ".healthStatus {width:10em}",
-                   ".healthReport {width:10em}");
+        ".healthReport {width:10em}");
   }
 
-  @Override protected Class<? extends SubView> content() {
+  @Override
+  protected Class<? extends SubView> content() {
     return NodesBlock.class;
   }
 
   private String nodesTableInit() {
     StringBuilder b = tableInit().append(", aoColumnDefs: [");
     b.append("{'bSearchable': false, 'aTargets': [ 6 ]}");
-    b.append(", {'sType': 'title-numeric', 'bSearchable': false, " +
-        "'aTargets': [ 7, 8 ] }");
+    b.append(", {'sType': 'title-numeric', 'bSearchable': false, "
+        + "'aTargets': [ 7, 8 ] }");
     b.append(", {'sType': 'title-numeric', 'aTargets': [ 4 ]}");
     b.append("]}");
     return b.toString();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a33b4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.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/RMWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
index 67c73b8..c0e6834 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
@@ -61,6 +61,7 @@ public class RMWebApp extends WebApp implements YarnWebParams {
     route(pajoin("/app", APPLICATION_ID), RmController.class, "app");
     route("/scheduler", RmController.class, "scheduler");
     route(pajoin("/queue", QUEUE_NAME), RmController.class, "queue");
+    route("/nodelabels", RmController.class, "nodelabels");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a33b4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.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/RmController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
index f186bf4..972432b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.util.StringHelper;
 import org.apache.hadoop.yarn.webapp.Controller;
-import org.apache.hadoop.yarn.webapp.WebAppException;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
 
 import com.google.inject.Inject;
@@ -93,4 +92,9 @@ public class RmController extends Controller {
   public void submit() {
     setTitle("Application Submission Not Allowed");
   }
+  
+  public void nodelabels() {
+    setTitle("Node Labels");
+    render(NodeLabelsPage.class);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a33b4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.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/MockNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
index 278c151..2d863d1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
@@ -30,11 +30,13 @@ import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
 /**
@@ -53,7 +55,12 @@ public class MockNodes {
           // One unhealthy node per rack.
           list.add(nodeInfo(i, perNode, NodeState.UNHEALTHY));
         }
-        list.add(newNodeInfo(i, perNode));
+        if (j == 0) {
+          // One node with label
+          list.add(nodeInfo(i, perNode, NodeState.RUNNING, ImmutableSet.of("x")));
+        } else {
+          list.add(newNodeInfo(i, perNode));
+        }
       }
     }
     return list;
@@ -100,10 +107,12 @@ public class MockNodes {
     private String healthReport;
     private long lastHealthReportTime;
     private NodeState state;
+    private Set<String> labels;
 
     public MockRMNodeImpl(NodeId nodeId, String nodeAddr, String httpAddress,
         Resource perNode, String rackName, String healthReport,
-        long lastHealthReportTime, int cmdPort, String hostName, NodeState state) {
+        long lastHealthReportTime, int cmdPort, String hostName, NodeState state,
+        Set<String> labels) {
       this.nodeId = nodeId;
       this.nodeAddr = nodeAddr;
       this.httpAddress = httpAddress;
@@ -114,6 +123,7 @@ public class MockNodes {
       this.cmdPort = cmdPort;
       this.hostName = hostName;
       this.state = state;
+      this.labels = labels;
     }
 
     @Override
@@ -207,16 +217,33 @@ public class MockNodes {
 
     @Override
     public Set<String> getNodeLabels() {
-      return RMNodeLabelsManager.EMPTY_STRING_SET;
+      if (labels != null) {
+        return labels;
+      }
+      return CommonNodeLabelsManager.EMPTY_STRING_SET;
     }
   };
 
-  private static RMNode buildRMNode(int rack, final Resource perNode, NodeState state, String httpAddr) {
-    return buildRMNode(rack, perNode, state, httpAddr, NODE_ID++, null, 123);
+  private static RMNode buildRMNode(int rack, final Resource perNode,
+      NodeState state, String httpAddr) {
+    return buildRMNode(rack, perNode, state, httpAddr, null);
   }
-
+  
+  private static RMNode buildRMNode(int rack, final Resource perNode,
+      NodeState state, String httpAddr, Set<String> labels) {
+    return buildRMNode(rack, perNode, state, httpAddr, NODE_ID++, null, 123,
+        labels);
+  }
+  
   private static RMNode buildRMNode(int rack, final Resource perNode,
       NodeState state, String httpAddr, int hostnum, String hostName, int port) {
+    return buildRMNode(rack, perNode, state, httpAddr, hostnum, hostName, port,
+        null);
+  }
+
+  private static RMNode buildRMNode(int rack, final Resource perNode,
+      NodeState state, String httpAddr, int hostnum, String hostName, int port,
+      Set<String> labels) {
     final String rackName = "rack"+ rack;
     final int nid = hostnum;
     final String nodeAddr = hostName + ":" + nid;
@@ -228,13 +255,18 @@ public class MockNodes {
     final String httpAddress = httpAddr;
     String healthReport = (state == NodeState.UNHEALTHY) ? null : "HealthyMe";
     return new MockRMNodeImpl(nodeID, nodeAddr, httpAddress, perNode,
-        rackName, healthReport, 0, nid, hostName, state);
+        rackName, healthReport, 0, nid, hostName, state, labels);
   }
 
   public static RMNode nodeInfo(int rack, final Resource perNode,
       NodeState state) {
     return buildRMNode(rack, perNode, state, "N/A");
   }
+  
+  public static RMNode nodeInfo(int rack, final Resource perNode,
+      NodeState state, Set<String> labels) {
+    return buildRMNode(rack, perNode, state, "N/A", labels);
+  }
 
   public static RMNode newNodeInfo(int rack, final Resource perNode) {
     return buildRMNode(rack, perNode, NodeState.RUNNING, "localhost:0");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a33b4a/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 842eaec..e21fcf9 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
@@ -839,7 +839,7 @@ public class TestWorkPreservingRMRestart {
   // Test if RM on recovery receives the container release request from AM
   // before it receives the container status reported by NM for recovery. this
   // container should not be recovered.
-  @Test (timeout = 30000)
+  @Test (timeout = 50000)
   public void testReleasedContainerNotRecovered() throws Exception {
     MemoryRMStateStore memStore = new MemoryRMStateStore();
     memStore.init(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a33b4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
index ed675f3..e4cdc71 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.nodelabels;
 
 import java.io.IOException;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
@@ -27,6 +28,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.nodelabels.NodeLabel;
 import org.apache.hadoop.yarn.nodelabels.NodeLabelTestBase;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.After;
@@ -428,4 +430,35 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
       Assert.fail("IOException from removeLabelsFromNode " + e);
     }
   }
+  
+  private void checkNodeLabelInfo(List<NodeLabel> infos, String labelName, int activeNMs, int memory) {
+    for (NodeLabel info : infos) {
+      if (info.getLabelName().equals(labelName)) {
+        Assert.assertEquals(activeNMs, info.getNumActiveNMs());
+        Assert.assertEquals(memory, info.getResource().getMemory());
+        return;
+      }
+    }
+    Assert.fail("Failed to find info has label=" + labelName);
+  }
+  
+  @Test(timeout = 5000)
+  public void testPullRMNodeLabelsInfo() throws IOException {
+    mgr.addToCluserNodeLabels(toSet("x", "y", "z"));
+    mgr.activateNode(NodeId.newInstance("n1", 1), Resource.newInstance(10, 0));
+    mgr.activateNode(NodeId.newInstance("n2", 1), Resource.newInstance(10, 0));
+    mgr.activateNode(NodeId.newInstance("n3", 1), Resource.newInstance(10, 0));
+    mgr.activateNode(NodeId.newInstance("n4", 1), Resource.newInstance(10, 0));
+    mgr.activateNode(NodeId.newInstance("n5", 1), Resource.newInstance(10, 0));
+    mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1"), toSet("x"),
+        toNodeId("n2"), toSet("x"), toNodeId("n3"), toSet("y")));
+    
+    // x, y, z and ""
+    List<NodeLabel> infos = mgr.pullRMNodeLabelsInfo();
+    Assert.assertEquals(4, infos.size());
+    checkNodeLabelInfo(infos, RMNodeLabelsManager.NO_LABEL, 2, 20);
+    checkNodeLabelInfo(infos, "x", 2, 20);
+    checkNodeLabelInfo(infos, "y", 1, 10);
+    checkNodeLabelInfo(infos, "z", 0, 0);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a33b4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.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/TestNodesPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java
index bb38079..62713cf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java
@@ -106,4 +106,49 @@ public class TestNodesPage {
             * numberOfActualTableHeaders + numberOfThInMetricsTable)).print(
         "<td");
   }
+  
+  @Test
+  public void testNodesBlockRenderForNodeLabelFilterWithNonEmptyLabel() {
+    NodesBlock nodesBlock = injector.getInstance(NodesBlock.class);
+    nodesBlock.set("node.label", "x");
+    nodesBlock.render();
+    PrintWriter writer = injector.getInstance(PrintWriter.class);
+    WebAppTests.flushOutput(injector);
+
+    Mockito.verify(
+        writer,
+        Mockito.times(numberOfRacks
+            * numberOfActualTableHeaders + numberOfThInMetricsTable)).print(
+        "<td");
+  }
+  
+  @Test
+  public void testNodesBlockRenderForNodeLabelFilterWithEmptyLabel() {
+    NodesBlock nodesBlock = injector.getInstance(NodesBlock.class);
+    nodesBlock.set("node.label", "");
+    nodesBlock.render();
+    PrintWriter writer = injector.getInstance(PrintWriter.class);
+    WebAppTests.flushOutput(injector);
+
+    Mockito.verify(
+        writer,
+        Mockito.times(numberOfRacks * (numberOfNodesPerRack - 1)
+            * numberOfActualTableHeaders + numberOfThInMetricsTable)).print(
+        "<td");
+  }
+  
+  @Test
+  public void testNodesBlockRenderForNodeLabelFilterWithAnyLabel() {
+    NodesBlock nodesBlock = injector.getInstance(NodesBlock.class);
+    nodesBlock.set("node.label", "*");
+    nodesBlock.render();
+    PrintWriter writer = injector.getInstance(PrintWriter.class);
+    WebAppTests.flushOutput(injector);
+
+    Mockito.verify(
+        writer,
+        Mockito.times(numberOfRacks * numberOfNodesPerRack
+            * numberOfActualTableHeaders + numberOfThInMetricsTable)).print(
+        "<td");
+  }
 }


[03/18] hadoop git commit: YARN-2492. Added node-labels page on RM web UI. Contributed by Wangda Tan

Posted by zh...@apache.org.
YARN-2492. Added node-labels page on RM web UI. Contributed by Wangda Tan


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

Branch: refs/heads/HDFS-EC
Commit: 756b66bcc38c3f4dcc878bf0e58f4a35cffeaea2
Parents: 5abc670
Author: Jian He <ji...@apache.org>
Authored: Tue Dec 30 15:35:45 2014 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:36 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   2 +
 .../nodelabels/CommonNodeLabelsManager.java     |  21 +--
 .../hadoop/yarn/nodelabels/NodeLabel.java       |  96 +++++++++++++
 .../hadoop/yarn/webapp/YarnWebParams.java       |   1 +
 .../nodelabels/RMNodeLabelsManager.java         |  36 +++--
 .../server/resourcemanager/webapp/NavBlock.java |   3 +-
 .../resourcemanager/webapp/NodeLabelsPage.java  |  91 ++++++++++++
 .../resourcemanager/webapp/NodesPage.java       | 139 ++++++++++---------
 .../server/resourcemanager/webapp/RMWebApp.java |   1 +
 .../resourcemanager/webapp/RmController.java    |   6 +-
 .../yarn/server/resourcemanager/MockNodes.java  |  46 +++++-
 .../TestWorkPreservingRMRestart.java            |   2 +-
 .../nodelabels/TestRMNodeLabelsManager.java     |  33 +++++
 .../resourcemanager/webapp/TestNodesPage.java   |  45 ++++++
 14 files changed, 421 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/756b66bc/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e707ea7..0ebf8a3 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -155,6 +155,8 @@ Release 2.7.0 - UNRELEASED
     YARN-2993. Several fixes (missing acl check, error log msg ...) and some 
     refinement in AdminService. (Yi Liu via junping_du)
 
+    YARN-2943. Added node-labels page on RM web UI. (Wangda Tan via jianhe)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/756b66bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
index 070aa1f..e888cc5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
@@ -72,8 +72,8 @@ public class CommonNodeLabelsManager extends AbstractService {
 
   protected Dispatcher dispatcher;
 
-  protected ConcurrentMap<String, Label> labelCollections =
-      new ConcurrentHashMap<String, Label>();
+  protected ConcurrentMap<String, NodeLabel> labelCollections =
+      new ConcurrentHashMap<String, NodeLabel>();
   protected ConcurrentMap<String, Host> nodeCollections =
       new ConcurrentHashMap<String, Host>();
 
@@ -82,19 +82,6 @@ public class CommonNodeLabelsManager extends AbstractService {
 
   protected NodeLabelsStore store;
 
-  protected static class Label {
-    private Resource resource;
-
-    protected Label() {
-      this.resource = Resource.newInstance(0, 0);
-    }
-
-    public Resource getResource() {
-      return this.resource;
-    }
-
-  }
-
   /**
    * A <code>Host</code> can have multiple <code>Node</code>s 
    */
@@ -201,7 +188,7 @@ public class CommonNodeLabelsManager extends AbstractService {
   protected void serviceInit(Configuration conf) throws Exception {
     initNodeLabelStore(conf);
     
-    labelCollections.put(NO_LABEL, new Label());
+    labelCollections.put(NO_LABEL, new NodeLabel(NO_LABEL));
   }
 
   protected void initNodeLabelStore(Configuration conf) throws Exception {
@@ -271,7 +258,7 @@ public class CommonNodeLabelsManager extends AbstractService {
     for (String label : labels) {
       // shouldn't overwrite it to avoid changing the Label.resource
       if (this.labelCollections.get(label) == null) {
-        this.labelCollections.put(label, new Label());
+        this.labelCollections.put(label, new NodeLabel(label));
         newLabels.add(label);
       }
     }

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/756b66bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
index 91d2a20..62c3c7a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
@@ -32,4 +32,5 @@ public interface YarnWebParams {
   String APP_STATE = "app.state";
   String QUEUE_NAME = "queue.name";
   String NODE_STATE = "node.state";
+  String NODE_LABEL = "node.label";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/756b66bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
index 646441a..828d1bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
@@ -19,10 +19,12 @@
 package org.apache.hadoop.yarn.server.resourcemanager.nodelabels;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -37,6 +39,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.nodelabels.NodeLabel;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeLabelsUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -360,8 +363,8 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         // no label in the past
         if (oldLabels.isEmpty()) {
           // update labels
-          Label label = labelCollections.get(NO_LABEL);
-          Resources.subtractFrom(label.getResource(), oldNM.resource);
+          NodeLabel label = labelCollections.get(NO_LABEL);
+          label.removeNode(oldNM.resource);
 
           // update queues, all queue can access this node
           for (Queue q : queueCollections.values()) {
@@ -370,11 +373,11 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         } else {
           // update labels
           for (String labelName : oldLabels) {
-            Label label = labelCollections.get(labelName);
+            NodeLabel label = labelCollections.get(labelName);
             if (null == label) {
               continue;
             }
-            Resources.subtractFrom(label.getResource(), oldNM.resource);
+            label.removeNode(oldNM.resource);
           }
 
           // update queues, only queue can access this node will be subtract
@@ -395,8 +398,8 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         // no label in the past
         if (newLabels.isEmpty()) {
           // update labels
-          Label label = labelCollections.get(NO_LABEL);
-          Resources.addTo(label.getResource(), newNM.resource);
+          NodeLabel label = labelCollections.get(NO_LABEL);
+          label.addNode(newNM.resource);
 
           // update queues, all queue can access this node
           for (Queue q : queueCollections.values()) {
@@ -405,8 +408,8 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         } else {
           // update labels
           for (String labelName : newLabels) {
-            Label label = labelCollections.get(labelName);
-            Resources.addTo(label.getResource(), newNM.resource);
+            NodeLabel label = labelCollections.get(labelName);
+            label.addNode(newNM.resource);
           }
 
           // update queues, only queue can access this node will be subtract
@@ -475,4 +478,21 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
   public void setRMContext(RMContext rmContext) {
     this.rmContext = rmContext;
   }
+
+  public List<NodeLabel> pullRMNodeLabelsInfo() {
+    try {
+      readLock.lock();
+      List<NodeLabel> infos = new ArrayList<NodeLabel>();
+
+      for (Entry<String, NodeLabel> entry : labelCollections.entrySet()) {
+        NodeLabel label = entry.getValue();
+        infos.add(label.getCopy());
+      }
+
+      Collections.sort(infos);
+      return infos;
+    } finally {
+      readLock.unlock();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/756b66bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.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/NavBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.java
index ce8fd9e..db00bb0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.java
@@ -33,7 +33,8 @@ public class NavBlock extends HtmlBlock {
         h3("Cluster").
         ul().
           li().a(url("cluster"), "About")._().
-          li().a(url("nodes"), "Nodes")._();
+          li().a(url("nodes"), "Nodes")._().
+          li().a(url("nodelabels"), "Node Labels")._();
     UL<LI<UL<DIV<Hamlet>>>> subAppsList = mainList.
           li().a(url("apps"), "Applications").
             ul();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/756b66bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java
new file mode 100644
index 0000000..5e8c1ed
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java
@@ -0,0 +1,91 @@
+/**
+ * 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;
+
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID;
+
+import org.apache.hadoop.yarn.nodelabels.NodeLabel;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.webapp.SubView;
+import org.apache.hadoop.yarn.webapp.YarnWebParams;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+
+import com.google.inject.Inject;
+
+public class NodeLabelsPage extends RmView {
+  static class NodeLabelsBlock extends HtmlBlock {
+    final ResourceManager rm;
+
+    @Inject
+    NodeLabelsBlock(ResourceManager rm, ViewContext ctx) {
+      super(ctx);
+      this.rm = rm;
+    }
+
+    @Override
+    protected void render(Block html) {
+      TBODY<TABLE<Hamlet>> tbody = html.table("#nodelabels").
+          thead().
+          tr().
+          th(".name", "Label Name").
+          th(".numOfActiveNMs", "Num Of Active NMs").
+          th(".totalResource", "Total Resource").
+          _()._().
+          tbody();
+  
+      RMNodeLabelsManager nlm = rm.getRMContext().getNodeLabelManager();
+      for (NodeLabel info : nlm.pullRMNodeLabelsInfo()) {
+        TR<TBODY<TABLE<Hamlet>>> row =
+            tbody.tr().td(
+                info.getLabelName().isEmpty() ? "<NO_LABEL>" : info
+                    .getLabelName());
+        int nActiveNMs = info.getNumActiveNMs();
+        if (nActiveNMs > 0) {
+          row = row.td()
+          .a(url("nodes",
+              "?" + YarnWebParams.NODE_LABEL + "=" + info.getLabelName()),
+              String.valueOf(nActiveNMs))
+           ._();
+        } else {
+          row = row.td(String.valueOf(nActiveNMs));
+        }
+        row.td(info.getResource().toString())._();
+      }
+      tbody._()._();
+    }
+  }
+
+  @Override protected void preHead(Page.HTML<_> html) {
+    commonPreHead(html);
+    String title = "Node labels of the cluster";
+    setTitle(title);
+    set(DATATABLES_ID, "nodelabels");
+    setTableStyles(html, "nodelabels", ".healthStatus {width:10em}",
+                   ".healthReport {width:10em}");
+  }
+
+  @Override protected Class<? extends SubView> content() {
+    return NodeLabelsBlock.class;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/756b66bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.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/NodesPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java
index d3849ae..f28a9a8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java
@@ -1,24 +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.
-*/
+ * 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;
 
-import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebApp.NODE_STATE;
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.NODE_STATE;
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.NODE_LABEL;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
@@ -28,7 +29,9 @@ import java.util.Collection;
 
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
@@ -60,26 +63,20 @@ class NodesPage extends RmView {
 
       ResourceScheduler sched = rm.getResourceScheduler();
       String type = $(NODE_STATE);
-      TBODY<TABLE<Hamlet>> tbody = html.table("#nodes").
-          thead().
-          tr().
-          th(".nodelabels", "Node Labels").
-          th(".rack", "Rack").
-          th(".state", "Node State").
-          th(".nodeaddress", "Node Address").
-          th(".nodehttpaddress", "Node HTTP Address").
-          th(".lastHealthUpdate", "Last health-update").
-          th(".healthReport", "Health-report").
-          th(".containers", "Containers").
-          th(".mem", "Mem Used").
-          th(".mem", "Mem Avail").
-          th(".vcores", "VCores Used").
-          th(".vcores", "VCores Avail").
-          th(".nodeManagerVersion", "Version").
-          _()._().
-          tbody();
+      String labelFilter = $(NODE_LABEL, CommonNodeLabelsManager.ANY).trim();
+      TBODY<TABLE<Hamlet>> tbody =
+          html.table("#nodes").thead().tr().th(".nodelabels", "Node Labels")
+              .th(".rack", "Rack").th(".state", "Node State")
+              .th(".nodeaddress", "Node Address")
+              .th(".nodehttpaddress", "Node HTTP Address")
+              .th(".lastHealthUpdate", "Last health-update")
+              .th(".healthReport", "Health-report")
+              .th(".containers", "Containers").th(".mem", "Mem Used")
+              .th(".mem", "Mem Avail").th(".vcores", "VCores Used")
+              .th(".vcores", "VCores Avail")
+              .th(".nodeManagerVersion", "Version")._()._().tbody();
       NodeState stateFilter = null;
-      if(type != null && !type.isEmpty()) {
+      if (type != null && !type.isEmpty()) {
         stateFilter = NodeState.valueOf(type.toUpperCase());
       }
       Collection<RMNode> rmNodes = this.rm.getRMContext().getRMNodes().values();
@@ -97,9 +94,9 @@ class NodesPage extends RmView {
         }
       }
       for (RMNode ni : rmNodes) {
-        if(stateFilter != null) {
+        if (stateFilter != null) {
           NodeState state = ni.getState();
-          if(!stateFilter.equals(state)) {
+          if (!stateFilter.equals(state)) {
             continue;
           }
         } else {
@@ -109,61 +106,71 @@ class NodesPage extends RmView {
             continue;
           }
         }
+        // Besides state, we need to filter label as well.
+        if (!labelFilter.equals(RMNodeLabelsManager.ANY)) {
+          if (labelFilter.isEmpty()) {
+            // Empty label filter means only shows nodes without label
+            if (!ni.getNodeLabels().isEmpty()) {
+              continue;
+            }
+          } else if (!ni.getNodeLabels().contains(labelFilter)) {
+            // Only nodes have given label can show on web page.
+            continue;
+          }
+        }
         NodeInfo info = new NodeInfo(ni, sched);
-        int usedMemory = (int)info.getUsedMemory();
-        int availableMemory = (int)info.getAvailableMemory();
-        TR<TBODY<TABLE<Hamlet>>> row = tbody.tr().
-            td(StringUtils.join(",", info.getNodeLabels())).
-            td(info.getRack()).
-            td(info.getState()).
-            td(info.getNodeId());
+        int usedMemory = (int) info.getUsedMemory();
+        int availableMemory = (int) info.getAvailableMemory();
+        TR<TBODY<TABLE<Hamlet>>> row =
+            tbody.tr().td(StringUtils.join(",", info.getNodeLabels()))
+                .td(info.getRack()).td(info.getState()).td(info.getNodeId());
         if (isInactive) {
           row.td()._("N/A")._();
         } else {
           String httpAddress = info.getNodeHTTPAddress();
-          row.td().a("//" + httpAddress,
-              httpAddress)._();
+          row.td().a("//" + httpAddress, httpAddress)._();
         }
-        row.td().br().$title(String.valueOf(info.getLastHealthUpdate()))._().
-              _(Times.format(info.getLastHealthUpdate()))._().
-            td(info.getHealthReport()).
-            td(String.valueOf(info.getNumContainers())).
-            td().br().$title(String.valueOf(usedMemory))._().
-              _(StringUtils.byteDesc(usedMemory * BYTES_IN_MB))._().
-            td().br().$title(String.valueOf(availableMemory))._().
-              _(StringUtils.byteDesc(availableMemory * BYTES_IN_MB))._().
-              td(String.valueOf(info.getUsedVirtualCores())).
-              td(String.valueOf(info.getAvailableVirtualCores())).
-            td(ni.getNodeManagerVersion()).
-            _();
+        row.td().br().$title(String.valueOf(info.getLastHealthUpdate()))._()
+            ._(Times.format(info.getLastHealthUpdate()))._()
+            .td(info.getHealthReport())
+            .td(String.valueOf(info.getNumContainers())).td().br()
+            .$title(String.valueOf(usedMemory))._()
+            ._(StringUtils.byteDesc(usedMemory * BYTES_IN_MB))._().td().br()
+            .$title(String.valueOf(availableMemory))._()
+            ._(StringUtils.byteDesc(availableMemory * BYTES_IN_MB))._()
+            .td(String.valueOf(info.getUsedVirtualCores()))
+            .td(String.valueOf(info.getAvailableVirtualCores()))
+            .td(ni.getNodeManagerVersion())._();
       }
       tbody._()._();
     }
   }
 
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override
+  protected void preHead(Page.HTML<_> html) {
     commonPreHead(html);
     String type = $(NODE_STATE);
     String title = "Nodes of the cluster";
-    if(type != null && !type.isEmpty()) {
-      title = title+" ("+type+")";
+    if (type != null && !type.isEmpty()) {
+      title = title + " (" + type + ")";
     }
     setTitle(title);
     set(DATATABLES_ID, "nodes");
     set(initID(DATATABLES, "nodes"), nodesTableInit());
     setTableStyles(html, "nodes", ".healthStatus {width:10em}",
-                   ".healthReport {width:10em}");
+        ".healthReport {width:10em}");
   }
 
-  @Override protected Class<? extends SubView> content() {
+  @Override
+  protected Class<? extends SubView> content() {
     return NodesBlock.class;
   }
 
   private String nodesTableInit() {
     StringBuilder b = tableInit().append(", aoColumnDefs: [");
     b.append("{'bSearchable': false, 'aTargets': [ 6 ]}");
-    b.append(", {'sType': 'title-numeric', 'bSearchable': false, " +
-        "'aTargets': [ 7, 8 ] }");
+    b.append(", {'sType': 'title-numeric', 'bSearchable': false, "
+        + "'aTargets': [ 7, 8 ] }");
     b.append(", {'sType': 'title-numeric', 'aTargets': [ 4 ]}");
     b.append("]}");
     return b.toString();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/756b66bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.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/RMWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
index 67c73b8..c0e6834 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
@@ -61,6 +61,7 @@ public class RMWebApp extends WebApp implements YarnWebParams {
     route(pajoin("/app", APPLICATION_ID), RmController.class, "app");
     route("/scheduler", RmController.class, "scheduler");
     route(pajoin("/queue", QUEUE_NAME), RmController.class, "queue");
+    route("/nodelabels", RmController.class, "nodelabels");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/756b66bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.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/RmController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
index f186bf4..972432b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.util.StringHelper;
 import org.apache.hadoop.yarn.webapp.Controller;
-import org.apache.hadoop.yarn.webapp.WebAppException;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
 
 import com.google.inject.Inject;
@@ -93,4 +92,9 @@ public class RmController extends Controller {
   public void submit() {
     setTitle("Application Submission Not Allowed");
   }
+  
+  public void nodelabels() {
+    setTitle("Node Labels");
+    render(NodeLabelsPage.class);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/756b66bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.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/MockNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
index 278c151..2d863d1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
@@ -30,11 +30,13 @@ import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
 /**
@@ -53,7 +55,12 @@ public class MockNodes {
           // One unhealthy node per rack.
           list.add(nodeInfo(i, perNode, NodeState.UNHEALTHY));
         }
-        list.add(newNodeInfo(i, perNode));
+        if (j == 0) {
+          // One node with label
+          list.add(nodeInfo(i, perNode, NodeState.RUNNING, ImmutableSet.of("x")));
+        } else {
+          list.add(newNodeInfo(i, perNode));
+        }
       }
     }
     return list;
@@ -100,10 +107,12 @@ public class MockNodes {
     private String healthReport;
     private long lastHealthReportTime;
     private NodeState state;
+    private Set<String> labels;
 
     public MockRMNodeImpl(NodeId nodeId, String nodeAddr, String httpAddress,
         Resource perNode, String rackName, String healthReport,
-        long lastHealthReportTime, int cmdPort, String hostName, NodeState state) {
+        long lastHealthReportTime, int cmdPort, String hostName, NodeState state,
+        Set<String> labels) {
       this.nodeId = nodeId;
       this.nodeAddr = nodeAddr;
       this.httpAddress = httpAddress;
@@ -114,6 +123,7 @@ public class MockNodes {
       this.cmdPort = cmdPort;
       this.hostName = hostName;
       this.state = state;
+      this.labels = labels;
     }
 
     @Override
@@ -207,16 +217,33 @@ public class MockNodes {
 
     @Override
     public Set<String> getNodeLabels() {
-      return RMNodeLabelsManager.EMPTY_STRING_SET;
+      if (labels != null) {
+        return labels;
+      }
+      return CommonNodeLabelsManager.EMPTY_STRING_SET;
     }
   };
 
-  private static RMNode buildRMNode(int rack, final Resource perNode, NodeState state, String httpAddr) {
-    return buildRMNode(rack, perNode, state, httpAddr, NODE_ID++, null, 123);
+  private static RMNode buildRMNode(int rack, final Resource perNode,
+      NodeState state, String httpAddr) {
+    return buildRMNode(rack, perNode, state, httpAddr, null);
   }
-
+  
+  private static RMNode buildRMNode(int rack, final Resource perNode,
+      NodeState state, String httpAddr, Set<String> labels) {
+    return buildRMNode(rack, perNode, state, httpAddr, NODE_ID++, null, 123,
+        labels);
+  }
+  
   private static RMNode buildRMNode(int rack, final Resource perNode,
       NodeState state, String httpAddr, int hostnum, String hostName, int port) {
+    return buildRMNode(rack, perNode, state, httpAddr, hostnum, hostName, port,
+        null);
+  }
+
+  private static RMNode buildRMNode(int rack, final Resource perNode,
+      NodeState state, String httpAddr, int hostnum, String hostName, int port,
+      Set<String> labels) {
     final String rackName = "rack"+ rack;
     final int nid = hostnum;
     final String nodeAddr = hostName + ":" + nid;
@@ -228,13 +255,18 @@ public class MockNodes {
     final String httpAddress = httpAddr;
     String healthReport = (state == NodeState.UNHEALTHY) ? null : "HealthyMe";
     return new MockRMNodeImpl(nodeID, nodeAddr, httpAddress, perNode,
-        rackName, healthReport, 0, nid, hostName, state);
+        rackName, healthReport, 0, nid, hostName, state, labels);
   }
 
   public static RMNode nodeInfo(int rack, final Resource perNode,
       NodeState state) {
     return buildRMNode(rack, perNode, state, "N/A");
   }
+  
+  public static RMNode nodeInfo(int rack, final Resource perNode,
+      NodeState state, Set<String> labels) {
+    return buildRMNode(rack, perNode, state, "N/A", labels);
+  }
 
   public static RMNode newNodeInfo(int rack, final Resource perNode) {
     return buildRMNode(rack, perNode, NodeState.RUNNING, "localhost:0");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/756b66bc/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 842eaec..e21fcf9 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
@@ -839,7 +839,7 @@ public class TestWorkPreservingRMRestart {
   // Test if RM on recovery receives the container release request from AM
   // before it receives the container status reported by NM for recovery. this
   // container should not be recovered.
-  @Test (timeout = 30000)
+  @Test (timeout = 50000)
   public void testReleasedContainerNotRecovered() throws Exception {
     MemoryRMStateStore memStore = new MemoryRMStateStore();
     memStore.init(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/756b66bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
index ed675f3..e4cdc71 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.nodelabels;
 
 import java.io.IOException;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
@@ -27,6 +28,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.nodelabels.NodeLabel;
 import org.apache.hadoop.yarn.nodelabels.NodeLabelTestBase;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.After;
@@ -428,4 +430,35 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
       Assert.fail("IOException from removeLabelsFromNode " + e);
     }
   }
+  
+  private void checkNodeLabelInfo(List<NodeLabel> infos, String labelName, int activeNMs, int memory) {
+    for (NodeLabel info : infos) {
+      if (info.getLabelName().equals(labelName)) {
+        Assert.assertEquals(activeNMs, info.getNumActiveNMs());
+        Assert.assertEquals(memory, info.getResource().getMemory());
+        return;
+      }
+    }
+    Assert.fail("Failed to find info has label=" + labelName);
+  }
+  
+  @Test(timeout = 5000)
+  public void testPullRMNodeLabelsInfo() throws IOException {
+    mgr.addToCluserNodeLabels(toSet("x", "y", "z"));
+    mgr.activateNode(NodeId.newInstance("n1", 1), Resource.newInstance(10, 0));
+    mgr.activateNode(NodeId.newInstance("n2", 1), Resource.newInstance(10, 0));
+    mgr.activateNode(NodeId.newInstance("n3", 1), Resource.newInstance(10, 0));
+    mgr.activateNode(NodeId.newInstance("n4", 1), Resource.newInstance(10, 0));
+    mgr.activateNode(NodeId.newInstance("n5", 1), Resource.newInstance(10, 0));
+    mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1"), toSet("x"),
+        toNodeId("n2"), toSet("x"), toNodeId("n3"), toSet("y")));
+    
+    // x, y, z and ""
+    List<NodeLabel> infos = mgr.pullRMNodeLabelsInfo();
+    Assert.assertEquals(4, infos.size());
+    checkNodeLabelInfo(infos, RMNodeLabelsManager.NO_LABEL, 2, 20);
+    checkNodeLabelInfo(infos, "x", 2, 20);
+    checkNodeLabelInfo(infos, "y", 1, 10);
+    checkNodeLabelInfo(infos, "z", 0, 0);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/756b66bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.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/TestNodesPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java
index bb38079..62713cf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java
@@ -106,4 +106,49 @@ public class TestNodesPage {
             * numberOfActualTableHeaders + numberOfThInMetricsTable)).print(
         "<td");
   }
+  
+  @Test
+  public void testNodesBlockRenderForNodeLabelFilterWithNonEmptyLabel() {
+    NodesBlock nodesBlock = injector.getInstance(NodesBlock.class);
+    nodesBlock.set("node.label", "x");
+    nodesBlock.render();
+    PrintWriter writer = injector.getInstance(PrintWriter.class);
+    WebAppTests.flushOutput(injector);
+
+    Mockito.verify(
+        writer,
+        Mockito.times(numberOfRacks
+            * numberOfActualTableHeaders + numberOfThInMetricsTable)).print(
+        "<td");
+  }
+  
+  @Test
+  public void testNodesBlockRenderForNodeLabelFilterWithEmptyLabel() {
+    NodesBlock nodesBlock = injector.getInstance(NodesBlock.class);
+    nodesBlock.set("node.label", "");
+    nodesBlock.render();
+    PrintWriter writer = injector.getInstance(PrintWriter.class);
+    WebAppTests.flushOutput(injector);
+
+    Mockito.verify(
+        writer,
+        Mockito.times(numberOfRacks * (numberOfNodesPerRack - 1)
+            * numberOfActualTableHeaders + numberOfThInMetricsTable)).print(
+        "<td");
+  }
+  
+  @Test
+  public void testNodesBlockRenderForNodeLabelFilterWithAnyLabel() {
+    NodesBlock nodesBlock = injector.getInstance(NodesBlock.class);
+    nodesBlock.set("node.label", "*");
+    nodesBlock.render();
+    PrintWriter writer = injector.getInstance(PrintWriter.class);
+    WebAppTests.flushOutput(injector);
+
+    Mockito.verify(
+        writer,
+        Mockito.times(numberOfRacks * numberOfNodesPerRack
+            * numberOfActualTableHeaders + numberOfThInMetricsTable)).print(
+        "<td");
+  }
 }


[14/18] hadoop git commit: HADOOP-11446 S3AOutputStream should use shared thread pool to avoid OutOfMemoryError

Posted by zh...@apache.org.
HADOOP-11446 S3AOutputStream should use shared thread pool to avoid OutOfMemoryError


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

Branch: refs/heads/HDFS-EC
Commit: e9e00ac6e81d7e61372d82af948df73d78fe67b9
Parents: d8b5117
Author: Steve Loughran <st...@apache.org>
Authored: Mon Jan 5 12:59:48 2015 +0000
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:38 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +
 .../org/apache/hadoop/fs/s3a/Constants.java     | 17 ++++
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 91 ++++++++++++++++++--
 .../apache/hadoop/fs/s3a/S3AOutputStream.java   | 13 +--
 4 files changed, 109 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e9e00ac6/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index ec75e8d..baf68d6 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -663,6 +663,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11039. ByteBufferReadable API doc is inconsistent with the
     implementations. (Yi Liu via Colin P. McCabe)
 
+    HADOOP-11446. S3AOutputStream should use shared thread pool to
+    avoid OutOfMemoryError. (Ted Yu via stevel)	
+    
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e9e00ac6/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 ee4bf68..f1b5d3d 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,23 @@ public class Constants {
   public static final String MAX_PAGING_KEYS = "fs.s3a.paging.maximum";
   public static final int DEFAULT_MAX_PAGING_KEYS = 5000;
 
+  // the maximum number of threads to allow in the pool used by TransferManager
+  public static final String MAX_THREADS = "fs.s3a.threads.max";
+  public static final int DEFAULT_MAX_THREADS = 256;
+
+  // the number of threads to keep in the pool used by TransferManager
+  public static final String CORE_THREADS = "fs.s3a.threads.core";
+  public static final int DEFAULT_CORE_THREADS = DEFAULT_MAXIMUM_CONNECTIONS;
+
+  // when the number of threads is greater than the core, this is the maximum time
+  // that excess idle threads will wait for new tasks before terminating.
+  public static final String KEEPALIVE_TIME = "fs.s3a.threads.keepalivetime";
+  public static final int DEFAULT_KEEPALIVE_TIME = 60;
+
+  // the maximum number of tasks that the LinkedBlockingQueue can hold
+  public static final String MAX_TOTAL_TASKS = "fs.s3a.max.total.tasks";
+  public static final int DEFAULT_MAX_TOTAL_TASKS = 1000;
+
   // size of each of or multipart pieces in bytes
   public static final String MULTIPART_SIZE = "fs.s3a.multipart.size";
   public static final long DEFAULT_MULTIPART_SIZE = 104857600; // 100 MB

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e9e00ac6/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 457351d..e6b1557 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
@@ -26,6 +26,11 @@ import java.net.URI;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.fs.s3.S3Credentials;
 
@@ -77,6 +82,7 @@ public class S3AFileSystem extends FileSystem {
   private String bucket;
   private int maxKeys;
   private long partSize;
+  private TransferManager transfers;
   private int partSizeThreshold;
   public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class);
   private CannedAccessControlList cannedACL;
@@ -85,6 +91,55 @@ public class S3AFileSystem extends FileSystem {
   // The maximum number of entries that can be deleted in any call to s3
   private static final int MAX_ENTRIES_TO_DELETE = 1000;
 
+  private static final AtomicInteger poolNumber = new AtomicInteger(1);
+  /**
+   * Returns a {@link java.util.concurrent.ThreadFactory} that names each created thread uniquely,
+   * with a common prefix.
+   * @param prefix The prefix of every created Thread's name
+   * @return a {@link java.util.concurrent.ThreadFactory} that names threads
+   */
+  public static ThreadFactory getNamedThreadFactory(final String prefix) {
+    SecurityManager s = System.getSecurityManager();
+    final ThreadGroup threadGroup = (s != null) ? s.getThreadGroup() : Thread.currentThread()
+        .getThreadGroup();
+
+    return new ThreadFactory() {
+      final AtomicInteger threadNumber = new AtomicInteger(1);
+      private final int poolNum = poolNumber.getAndIncrement();
+      final ThreadGroup group = threadGroup;
+
+      @Override
+      public Thread newThread(Runnable r) {
+        final String name = prefix + "-pool" + poolNum + "-t" + threadNumber.getAndIncrement();
+        return new Thread(group, r, name);
+      }
+    };
+  }
+
+  /**
+   * Get a named {@link ThreadFactory} that just builds daemon threads.
+   * @param prefix name prefix for all threads created from the factory
+   * @return a thread factory that creates named, daemon threads with
+   *         the supplied exception handler and normal priority
+   */
+  private static ThreadFactory newDaemonThreadFactory(final String prefix) {
+    final ThreadFactory namedFactory = getNamedThreadFactory(prefix);
+    return new ThreadFactory() {
+      @Override
+      public Thread newThread(Runnable r) {
+        Thread t = namedFactory.newThread(r);
+        if (!t.isDaemon()) {
+          t.setDaemon(true);
+        }
+        if (t.getPriority() != Thread.NORM_PRIORITY) {
+          t.setPriority(Thread.NORM_PRIORITY);
+        }
+        return t;
+      }
+
+    };
+  }
+
   /** Called after a new FileSystem instance is constructed.
    * @param name a uri whose authority section names the host, port, etc.
    *   for this FileSystem
@@ -93,7 +148,6 @@ public class S3AFileSystem extends FileSystem {
   public void initialize(URI name, Configuration conf) throws IOException {
     super.initialize(name, conf);
 
-
     uri = URI.create(name.getScheme() + "://" + name.getAuthority());
     workingDir = new Path("/user", System.getProperty("user.name")).makeQualified(this.uri,
         this.getWorkingDirectory());
@@ -138,6 +192,34 @@ public class S3AFileSystem extends FileSystem {
       partSizeThreshold = 5 * 1024 * 1024;
     }
 
+    int maxThreads = conf.getInt(MAX_THREADS, DEFAULT_MAX_THREADS);
+    int coreThreads = conf.getInt(CORE_THREADS, DEFAULT_CORE_THREADS);
+    if (maxThreads == 0) {
+      maxThreads = Runtime.getRuntime().availableProcessors() * 8;
+    }
+    if (coreThreads == 0) {
+      coreThreads = Runtime.getRuntime().availableProcessors() * 8;
+    }
+    long keepAliveTime = conf.getLong(KEEPALIVE_TIME, DEFAULT_KEEPALIVE_TIME);
+    LinkedBlockingQueue<Runnable> workQueue =
+      new LinkedBlockingQueue<Runnable>(maxThreads *
+        conf.getInt(MAX_TOTAL_TASKS, DEFAULT_MAX_TOTAL_TASKS));
+    ThreadPoolExecutor tpe = new ThreadPoolExecutor(
+        coreThreads,
+        maxThreads,
+        keepAliveTime,
+        TimeUnit.SECONDS,
+        workQueue,
+        newDaemonThreadFactory("s3a-transfer-shared-"));
+    tpe.allowCoreThreadTimeOut(true);
+
+    TransferManagerConfiguration transferConfiguration = new TransferManagerConfiguration();
+    transferConfiguration.setMinimumUploadPartSize(partSize);
+    transferConfiguration.setMultipartUploadThreshold(partSizeThreshold);
+
+    transfers = new TransferManager(s3, tpe);
+    transfers.setConfiguration(transferConfiguration);
+
     String cannedACLName = conf.get(CANNED_ACL, DEFAULT_CANNED_ACL);
     if (!cannedACLName.isEmpty()) {
       cannedACL = CannedAccessControlList.valueOf(cannedACLName);
@@ -155,11 +237,10 @@ public class S3AFileSystem extends FileSystem {
       DEFAULT_PURGE_EXISTING_MULTIPART_AGE);
 
     if (purgeExistingMultipart) {
-      TransferManager transferManager = new TransferManager(s3);
       Date purgeBefore = new Date(new Date().getTime() - purgeExistingMultipartAge*1000);
 
-      transferManager.abortMultipartUploads(bucket, purgeBefore);
-      transferManager.shutdownNow(false);
+      transfers.abortMultipartUploads(bucket, purgeBefore);
+      transfers.shutdownNow(false);
     }
 
     serverSideEncryptionAlgorithm = conf.get(SERVER_SIDE_ENCRYPTION_ALGORITHM);
@@ -245,7 +326,7 @@ public class S3AFileSystem extends FileSystem {
     }
 
     // We pass null to FSDataOutputStream so it won't count writes that are being buffered to a file
-    return new FSDataOutputStream(new S3AOutputStream(getConf(), s3, this, 
+    return new FSDataOutputStream(new S3AOutputStream(getConf(), transfers, this,
       bucket, key, progress, cannedACL, statistics, 
       serverSideEncryptionAlgorithm), null);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e9e00ac6/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
index 7783b99..2b611b6 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
@@ -49,7 +49,7 @@ public class S3AOutputStream extends OutputStream {
   private boolean closed;
   private String key;
   private String bucket;
-  private AmazonS3Client client;
+  private TransferManager transfers;
   private Progressable progress;
   private long partSize;
   private int partSizeThreshold;
@@ -61,14 +61,14 @@ public class S3AOutputStream extends OutputStream {
 
   public static final Logger LOG = S3AFileSystem.LOG;
 
-  public S3AOutputStream(Configuration conf, AmazonS3Client client, 
+  public S3AOutputStream(Configuration conf, TransferManager transfers,
     S3AFileSystem fs, String bucket, String key, Progressable progress, 
     CannedAccessControlList cannedACL, FileSystem.Statistics statistics, 
     String serverSideEncryptionAlgorithm)
       throws IOException {
     this.bucket = bucket;
     this.key = key;
-    this.client = client;
+    this.transfers = transfers;
     this.progress = progress;
     this.fs = fs;
     this.cannedACL = cannedACL;
@@ -114,13 +114,6 @@ public class S3AOutputStream extends OutputStream {
 
 
     try {
-      TransferManagerConfiguration transferConfiguration = new TransferManagerConfiguration();
-      transferConfiguration.setMinimumUploadPartSize(partSize);
-      transferConfiguration.setMultipartUploadThreshold(partSizeThreshold);
-
-      TransferManager transfers = new TransferManager(client);
-      transfers.setConfiguration(transferConfiguration);
-
       final ObjectMetadata om = new ObjectMetadata();
       if (StringUtils.isNotBlank(serverSideEncryptionAlgorithm)) {
         om.setServerSideEncryption(serverSideEncryptionAlgorithm);


[09/18] hadoop git commit: HADOOP-10788. Rewrite kms to use new shell framework (John Smith via aw)

Posted by zh...@apache.org.
HADOOP-10788. Rewrite kms to use new shell framework (John Smith via aw)


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

Branch: refs/heads/HDFS-EC
Commit: 80f386d1e50a4158c351e5518b6998f871b1383d
Parents: de378cb
Author: Allen Wittenauer <aw...@apache.org>
Authored: Fri Jan 2 10:52:23 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:37 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   2 +
 .../src/main/bin/hadoop-functions.sh            |  19 ++
 .../hadoop-kms/src/main/conf/kms-env.sh         |  45 +++-
 .../hadoop-kms/src/main/libexec/kms-config.sh   | 207 ++++---------------
 .../hadoop-kms/src/main/sbin/kms.sh             | 119 ++++++-----
 5 files changed, 171 insertions(+), 221 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/80f386d1/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 793e583..d7ebeac 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -141,6 +141,8 @@ Trunk (Unreleased)
     HADOOP-11352 Clean up test-patch.sh to disable "+1 contrib tests"
     (Akira AJISAKA via stevel)
 
+    HADOOP-10788. Rewrite kms to use new shell framework (John Smith via aw)
+
   BUG FIXES
 
     HADOOP-9451. Fault single-layer config if node group topology is enabled.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80f386d1/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
index a953e0c..799aad0 100644
--- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
+++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
@@ -666,6 +666,25 @@ function hadoop_finalize_classpath
   hadoop_add_to_classpath_userpath
 }
 
+function hadoop_finalize_catalina_opts
+{
+
+  local prefix=${HADOOP_CATALINA_PREFIX}
+
+  hadoop_add_param CATALINA_OPTS hadoop.home.dir "-Dhadoop.home.dir=${HADOOP_PREFIX}"
+  if [[ -n "${JAVA_LIBRARY_PATH}" ]]; then
+    hadoop_add_param CATALINA_OPTS java.library.path "-Djava.library.path=${JAVA_LIBRARY_PATH}"
+  fi
+  hadoop_add_param CATALINA_OPTS "${prefix}.home.dir" "-D${prefix}.home.dir=${HADOOP_PREFIX}"
+  hadoop_add_param CATALINA_OPTS "${prefix}.config.dir" "-D${prefix}.config.dir=${HADOOP_CATALINA_CONFIG}"
+  hadoop_add_param CATALINA_OPTS "${prefix}.log.dir" "-D${prefix}.log.dir=${HADOOP_CATALINA_LOG}"
+  hadoop_add_param CATALINA_OPTS "${prefix}.temp.dir" "-D${prefix}.temp.dir=${HADOOP_CATALINA_TEMP}"
+  hadoop_add_param CATALINA_OPTS "${prefix}.admin.port" "-D${prefix}.admin.port=${HADOOP_CATALINA_ADMIN_PORT}"
+  hadoop_add_param CATALINA_OPTS "${prefix}.http.port" "-D${prefix}.http.port=${HADOOP_CATALINA_HTTP_PORT}"
+  hadoop_add_param CATALINA_OPTS "${prefix}.max.threads" "-D${prefix}.max.threads=${HADOOP_CATALINA_MAX_THREADS}"
+  hadoop_add_param CATALINA_OPTS "${prefix}.ssl.keystore.file" "-D${prefix}.ssl.keystore.file=${HADOOP_CATALINA_SSL_KEYSTORE_FILE}"
+}
+
 function hadoop_finalize
 {
   # user classpath gets added at the last minute. this allows

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80f386d1/hadoop-common-project/hadoop-kms/src/main/conf/kms-env.sh
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/conf/kms-env.sh b/hadoop-common-project/hadoop-kms/src/main/conf/kms-env.sh
index 44dfe6a..7045379 100644
--- a/hadoop-common-project/hadoop-kms/src/main/conf/kms-env.sh
+++ b/hadoop-common-project/hadoop-kms/src/main/conf/kms-env.sh
@@ -14,19 +14,17 @@
 #
 
 # Set kms specific environment variables here.
-
-# Settings for the Embedded Tomcat that runs KMS
-# Java System properties for KMS should be specified in this variable
 #
-# export CATALINA_OPTS=
+# hadoop-env.sh is read prior to this file.
+#
 
 # KMS logs directory
 #
-# export KMS_LOG=${KMS_HOME}/logs
+# export KMS_LOG=${HADOOP_LOG_DIR}
 
 # KMS temporary directory
 #
-# export KMS_TEMP=${KMS_HOME}/temp
+# export KMS_TEMP=${HADOOP_PREFIX}/temp
 
 # The HTTP port used by KMS
 #
@@ -34,7 +32,7 @@
 
 # The Admin port used by KMS
 #
-# export KMS_ADMIN_PORT=`expr ${KMS_HTTP_PORT} + 1`
+# export KMS_ADMIN_PORT=$((KMS_HTTP_PORT + 1))
 
 # The maximum number of Tomcat handler threads
 #
@@ -44,12 +42,37 @@
 #
 # export KMS_SSL_KEYSTORE_FILE=${HOME}/.keystore
 
+#
 # The password of the SSL keystore if using SSL
 #
 # export KMS_SSL_KEYSTORE_PASS=password
 
-# The full path to any native libraries that need to be loaded
-# (For eg. location of natively compiled tomcat Apache portable
-# runtime (APR) libraries
 #
-# export JAVA_LIBRARY_PATH=${HOME}/lib/native
+# The password of the truststore
+#
+# export KMS_SSL_TRUSTSTORE_PASS=
+
+
+##
+## Tomcat specific settings
+##
+#
+# Location of tomcat
+#
+# export KMS_CATALINA_HOME=${HADOOP_PREFIX}/share/hadoop/kms/tomcat
+
+# Java System properties for KMS should be specified in this variable.
+# The java.library.path and hadoop.home.dir properties are automatically
+# configured.  In order to supplement java.library.path,
+# one should add to the JAVA_LIBRARY_PATH env var.
+#
+# export CATALINA_OPTS=
+
+# PID file
+#
+# export CATALINA_PID=${HADOOP_PID_DIR}/hadoop-${HADOOP_IDENT_STRING}-kms.pid
+
+# Output file
+#
+# export CATALINA_OUT=${KMS_LOG}/hadoop-${HADOOP_IDENT_STRING}-kms-${HOSTNAME}.out
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80f386d1/hadoop-common-project/hadoop-kms/src/main/libexec/kms-config.sh
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/libexec/kms-config.sh b/hadoop-common-project/hadoop-kms/src/main/libexec/kms-config.sh
index 3ac929a..7cfb78d 100644
--- a/hadoop-common-project/hadoop-kms/src/main/libexec/kms-config.sh
+++ b/hadoop-common-project/hadoop-kms/src/main/libexec/kms-config.sh
@@ -13,182 +13,63 @@
 #  limitations under the License.
 #
 
-# resolve links - $0 may be a softlink
-PRG="${0}"
-
-while [ -h "${PRG}" ]; do
-  ls=`ls -ld "${PRG}"`
-  link=`expr "$ls" : '.*-> \(.*\)$'`
-  if expr "$link" : '/.*' > /dev/null; then
-    PRG="$link"
-  else
-    PRG=`dirname "${PRG}"`/"$link"
+function hadoop_subproject_init
+{
+  local this
+  local binparent
+  local varlist
+
+  if [[ -z "${HADOOP_KMS_ENV_PROCESSED}" ]]; then
+    if [[ -e "${HADOOP_CONF_DIR}/kms-env.sh" ]]; then
+      . "${HADOOP_CONF_DIR}/kms-env.sh"
+      export HADOOP_KMS_ENV_PROCESSED=true
+    fi
   fi
-done
-
-BASEDIR=`dirname ${PRG}`
-BASEDIR=`cd ${BASEDIR}/..;pwd`
-
-
-function print() {
-  if [ "${KMS_SILENT}" != "true" ]; then
-    echo "$@"
-  fi
-}
-
-# if KMS_HOME is already set warn it will be ignored
-#
-if [ "${KMS_HOME}" != "" ]; then
-  echo "WARNING: current setting of KMS_HOME ignored"
-fi
 
-print
+  export HADOOP_CATALINA_PREFIX=kms
 
-# setting KMS_HOME to the installation dir, it cannot be changed
-#
-export KMS_HOME=${BASEDIR}
-kms_home=${KMS_HOME}
-print "Setting KMS_HOME:          ${KMS_HOME}"
+  export HADOOP_CATALINA_TEMP="${KMS_TEMP:-${HADOOP_PREFIX}/temp}"
 
-# if the installation has a env file, source it
-# this is for native packages installations
-#
-if [ -e "${KMS_HOME}/bin/kms-env.sh" ]; then
-  print "Sourcing:                    ${KMS_HOME}/bin/kms-env.sh"
-  source ${KMS_HOME}/bin/kms-env.sh
-  grep "^ *export " ${KMS_HOME}/bin/kms-env.sh | sed 's/ *export/  setting/'
-fi
-
-# verify that the sourced env file didn't change KMS_HOME
-# if so, warn and revert
-#
-if [ "${KMS_HOME}" != "${kms_home}" ]; then
-  print "WARN: KMS_HOME resetting to ''${KMS_HOME}'' ignored"
-  export KMS_HOME=${kms_home}
-  print "  using KMS_HOME:        ${KMS_HOME}"
-fi
+  export HADOOP_CONF_DIR="${KMS_CONFIG:-${HADOOP_CONF_DIR}}"
+  export HADOOP_CATALINA_CONFIG="${HADOOP_CONF_DIR}"
 
-if [ "${KMS_CONFIG}" = "" ]; then
-  export KMS_CONFIG=${KMS_HOME}/etc/hadoop
-  print "Setting KMS_CONFIG:        ${KMS_CONFIG}"
-else
-  print "Using   KMS_CONFIG:        ${KMS_CONFIG}"
-fi
-kms_config=${KMS_CONFIG}
+  export HADOOP_LOG_DIR="${KMS_LOG:-${HADOOP_LOG_DIR}}"
+  export HADOOP_CATALINA_LOG="${HADOOP_LOG_DIR}"
 
-# if the configuration dir has a env file, source it
-#
-if [ -e "${KMS_CONFIG}/kms-env.sh" ]; then
-  print "Sourcing:                    ${KMS_CONFIG}/kms-env.sh"
-  source ${KMS_CONFIG}/kms-env.sh
-  grep "^ *export " ${KMS_CONFIG}/kms-env.sh | sed 's/ *export/  setting/'
-fi
+  export HADOOP_CATALINA_HTTP_PORT="${KMS_HTTP_PORT:-16000}"
+  export HADOOP_CATALINA_ADMIN_PORT="${KMS_ADMIN_PORT:-$((HADOOP_CATALINA_HTTP_PORT+1))}"
+  export HADOOP_CATALINA_MAX_THREADS="${KMS_MAX_THREADS:-1000}"
 
-# verify that the sourced env file didn't change KMS_HOME
-# if so, warn and revert
-#
-if [ "${KMS_HOME}" != "${kms_home}" ]; then
-  echo "WARN: KMS_HOME resetting to ''${KMS_HOME}'' ignored"
-  export KMS_HOME=${kms_home}
-fi
+  export HADOOP_CATALINA_SSL_KEYSTORE_FILE="${KMS_SSL_KEYSTORE_FILE:-${HOME}/.keystore}"
 
-# verify that the sourced env file didn't change KMS_CONFIG
-# if so, warn and revert
-#
-if [ "${KMS_CONFIG}" != "${kms_config}" ]; then
-  echo "WARN: KMS_CONFIG resetting to ''${KMS_CONFIG}'' ignored"
-  export KMS_CONFIG=${kms_config}
-fi
+  # this is undocumented, but older versions would rip the TRUSTSTORE_PASS out of the
+  # CATALINA_OPTS
+  # shellcheck disable=SC2086
+  export KMS_SSL_TRUSTSTORE_PASS=${KMS_SSL_TRUSTSTORE_PASS:-"$(echo ${CATALINA_OPTS} | grep -o 'trustStorePassword=[^ ]*' | cut -f2 -d= )"}
 
-if [ "${KMS_LOG}" = "" ]; then
-  export KMS_LOG=${KMS_HOME}/logs
-  print "Setting KMS_LOG:           ${KMS_LOG}"
-else
-  print "Using   KMS_LOG:           ${KMS_LOG}"
-fi
+  export CATALINA_BASE="${CATALINA_BASE:-${HADOOP_PREFIX}/share/hadoop/kms/tomcat}"
+  export HADOOP_CATALINA_HOME="${KMS_CATALINA_HOME:-${CATALINA_BASE}}"
 
-if [ ! -f ${KMS_LOG} ]; then
-  mkdir -p ${KMS_LOG}
-fi
+  export CATALINA_OUT="${CATALINA_OUT:-${HADOOP_LOG_DIR}/hadoop-${HADOOP_IDENT_STRING}-kms-${HOSTNAME}.out}"
 
-if [ "${KMS_TEMP}" = "" ]; then
-  export KMS_TEMP=${KMS_HOME}/temp
-  print "Setting KMS_TEMP:           ${KMS_TEMP}"
-else
-  print "Using   KMS_TEMP:           ${KMS_TEMP}"
-fi
+  export CATALINA_PID="${CATALINA_PID:-${HADOOP_PID_DIR}/hadoop-${HADOOP_IDENT_STRING}-kms.pid}"
 
-if [ ! -f ${KMS_TEMP} ]; then
-  mkdir -p ${KMS_TEMP}
-fi
-
-if [ "${KMS_HTTP_PORT}" = "" ]; then
-  export KMS_HTTP_PORT=16000
-  print "Setting KMS_HTTP_PORT:     ${KMS_HTTP_PORT}"
-else
-  print "Using   KMS_HTTP_PORT:     ${KMS_HTTP_PORT}"
-fi
-
-if [ "${KMS_ADMIN_PORT}" = "" ]; then
-  export KMS_ADMIN_PORT=`expr $KMS_HTTP_PORT +  1`
-  print "Setting KMS_ADMIN_PORT:     ${KMS_ADMIN_PORT}"
-else
-  print "Using   KMS_ADMIN_PORT:     ${KMS_ADMIN_PORT}"
-fi
-
-if [ "${KMS_MAX_THREADS}" = "" ]; then
-  export KMS_MAX_THREADS=1000
-  print "Setting KMS_MAX_THREADS:     ${KMS_MAX_THREADS}"
-else
-  print "Using   KMS_MAX_THREADS:     ${KMS_MAX_THREADS}"
-fi
-
-if [ "${KMS_SSL_KEYSTORE_FILE}" = "" ]; then
-  export KMS_SSL_KEYSTORE_FILE=${HOME}/.keystore
-  print "Setting KMS_SSL_KEYSTORE_FILE:     ${KMS_SSL_KEYSTORE_FILE}"
-else
-  print "Using   KMS_SSL_KEYSTORE_FILE:     ${KMS_SSL_KEYSTORE_FILE}"
-fi
-
-# If KMS_SSL_KEYSTORE_PASS is explicitly set to ""
-# then reset to "password". DO NOT set to "password" if
-# variable is NOT defined.
-if [ "${KMS_SSL_KEYSTORE_PASS}" = "" ]; then
-  if [ -n "${KMS_SSL_KEYSTORE_PASS+1}" ]; then
-    export KMS_SSL_KEYSTORE_PASS=password
-    print "Setting KMS_SSL_KEYSTORE_PASS:     ********"
+  if [[ -n "${HADOOP_SHELL_SCRIPT_DEBUG}" ]]; then
+    varlist=$(env | egrep '(^KMS|^CATALINA)' | cut -f1 -d= | grep -v _PASS)
+    for i in ${varlist}; do
+      hadoop_debug "Setting ${i} to ${!i}"
+    done
   fi
-else
-  KMS_SSL_KEYSTORE_PASS_DISP=`echo ${KMS_SSL_KEYSTORE_PASS} | sed 's/./*/g'`
-  print "Using   KMS_SSL_KEYSTORE_PASS:     ${KMS_SSL_KEYSTORE_PASS_DISP}"
-fi
-
-if [ "${CATALINA_BASE}" = "" ]; then
-  export CATALINA_BASE=${KMS_HOME}/share/hadoop/kms/tomcat
-  print "Setting CATALINA_BASE:       ${CATALINA_BASE}"
-else
-  print "Using   CATALINA_BASE:       ${CATALINA_BASE}"
-fi
-
-if [ "${KMS_CATALINA_HOME}" = "" ]; then
-  export KMS_CATALINA_HOME=${CATALINA_BASE}
-  print "Setting KMS_CATALINA_HOME:       ${KMS_CATALINA_HOME}"
-else
-  print "Using   KMS_CATALINA_HOME:       ${KMS_CATALINA_HOME}"
-fi
-
-if [ "${CATALINA_OUT}" = "" ]; then
-  export CATALINA_OUT=${KMS_LOG}/kms-catalina.out
-  print "Setting CATALINA_OUT:        ${CATALINA_OUT}"
-else
-  print "Using   CATALINA_OUT:        ${CATALINA_OUT}"
-fi
+}
 
-if [ "${CATALINA_PID}" = "" ]; then
-  export CATALINA_PID=/tmp/kms.pid
-  print "Setting CATALINA_PID:        ${CATALINA_PID}"
+if [[ -n "${HADOOP_COMMON_HOME}" ]] &&
+   [[ -e "${HADOOP_COMMON_HOME}/libexec/hadoop-config.sh" ]]; then
+  . "${HADOOP_COMMON_HOME}/libexec/hadoop-config.sh"
+elif [[ -e "${HADOOP_LIBEXEC_DIR}/hadoop-config.sh" ]]; then
+  . "${HADOOP_LIBEXEC_DIR}/hadoop-config.sh"
+elif [[ -e "${HADOOP_PREFIX}/libexec/hadoop-config.sh" ]]; then
+  . "${HADOOP_PREFIX}/libexec/hadoop-config.sh"
 else
-  print "Using   CATALINA_PID:        ${CATALINA_PID}"
+  echo "ERROR: Hadoop common not found." 2>&1
+  exit 1
 fi
-
-print

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80f386d1/hadoop-common-project/hadoop-kms/src/main/sbin/kms.sh
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/sbin/kms.sh b/hadoop-common-project/hadoop-kms/src/main/sbin/kms.sh
index f6ef6a5..e4d4f93 100644
--- a/hadoop-common-project/hadoop-kms/src/main/sbin/kms.sh
+++ b/hadoop-common-project/hadoop-kms/src/main/sbin/kms.sh
@@ -13,32 +13,39 @@
 #  limitations under the License.
 #
 
-# resolve links - $0 may be a softlink
-PRG="${0}"
-
-while [ -h "${PRG}" ]; do
-  ls=`ls -ld "${PRG}"`
-  link=`expr "$ls" : '.*-> \(.*\)$'`
-  if expr "$link" : '/.*' > /dev/null; then
-    PRG="$link"
-  else
-    PRG=`dirname "${PRG}"`/"$link"
-  fi
-done
-
-BASEDIR=`dirname ${PRG}`
-BASEDIR=`cd ${BASEDIR}/..;pwd`
-
-KMS_SILENT=${KMS_SILENT:-true}
-
-HADOOP_LIBEXEC_DIR="${HADOOP_LIBEXEC_DIR:-${BASEDIR}/libexec}"
-source ${HADOOP_LIBEXEC_DIR}/kms-config.sh
+function hadoop_usage()
+{
+  echo "Usage: kms.sh [--config confdir] [--debug] --daemon start|status|stop"
+  echo "       kms.sh [--config confdir] [--debug] COMMAND"
+  echo "            where COMMAND is one of:"
+  echo "  run               Start kms in the current window"
+  echo "  run -security     Start in the current window with security manager"
+  echo "  start             Start kms in a separate window"
+  echo "  start -security   Start in a separate window with security manager"
+  echo "  status            Return the LSB compliant status"
+  echo "  stop              Stop kms, waiting up to 5 seconds for the process to end"
+  echo "  stop n            Stop kms, waiting up to n seconds for the process to end"
+  echo "  stop -force       Stop kms, wait up to 5 seconds and then use kill -KILL if still running"
+  echo "  stop n -force     Stop kms, wait up to n seconds and then use kill -KILL if still running"
+}
 
+# let's locate libexec...
+if [[ -n "${HADOOP_PREFIX}" ]]; then
+  DEFAULT_LIBEXEC_DIR="${HADOOP_PREFIX}/libexec"
+else
+  this="${BASH_SOURCE-$0}"
+  bin=$(cd -P -- "$(dirname -- "${this}")" >/dev/null && pwd -P)
+  DEFAULT_LIBEXEC_DIR="${bin}/../libexec"
+fi
 
-if [ "x$JAVA_LIBRARY_PATH" = "x" ]; then
-  JAVA_LIBRARY_PATH="${HADOOP_LIBEXEC_DIR}/../lib/native/"
+HADOOP_LIBEXEC_DIR="${HADOOP_LIBEXEC_DIR:-$DEFAULT_LIBEXEC_DIR}"
+# shellcheck disable=SC2034
+HADOOP_NEW_CONFIG=true
+if [[ -f "${HADOOP_LIBEXEC_DIR}/kms-config.sh" ]]; then
+  . "${HADOOP_LIBEXEC_DIR}/kms-config.sh"
 else
-  JAVA_LIBRARY_PATH="${HADOOP_LIBEXEC_DIR}/../lib/native/:${JAVA_LIBRARY_PATH}"
+  echo "ERROR: Cannot execute ${HADOOP_LIBEXEC_DIR}/kms-config.sh." 2>&1
+  exit 1
 fi
 
 # The Java System property 'kms.http.port' it is not used by Kms,
@@ -46,38 +53,56 @@ fi
 #
 
 # Mask the trustStorePassword
-KMS_SSL_TRUSTSTORE_PASS=`echo $CATALINA_OPTS | grep -o 'trustStorePassword=[^ ]*' | awk -F'=' '{print $2}'`
-CATALINA_OPTS_DISP=`echo ${CATALINA_OPTS} | sed -e 's/trustStorePassword=[^ ]*/trustStorePassword=***/'`
-print "Using   CATALINA_OPTS:       ${CATALINA_OPTS_DISP}"
+# shellcheck disable=SC2086
+CATALINA_OPTS_DISP="$(echo ${CATALINA_OPTS} | sed -e 's/trustStorePassword=[^ ]*/trustStorePassword=***/')"
 
-catalina_opts="-Dkms.home.dir=${KMS_HOME}";
-catalina_opts="${catalina_opts} -Dkms.config.dir=${KMS_CONFIG}";
-catalina_opts="${catalina_opts} -Dkms.log.dir=${KMS_LOG}";
-catalina_opts="${catalina_opts} -Dkms.temp.dir=${KMS_TEMP}";
-catalina_opts="${catalina_opts} -Dkms.admin.port=${KMS_ADMIN_PORT}";
-catalina_opts="${catalina_opts} -Dkms.http.port=${KMS_HTTP_PORT}";
-catalina_opts="${catalina_opts} -Dkms.max.threads=${KMS_MAX_THREADS}";
-catalina_opts="${catalina_opts} -Dkms.ssl.keystore.file=${KMS_SSL_KEYSTORE_FILE}";
-catalina_opts="${catalina_opts} -Djava.library.path=${JAVA_LIBRARY_PATH}";
+hadoop_debug "Using   CATALINA_OPTS:       ${CATALINA_OPTS_DISP}"
 
-print "Adding to CATALINA_OPTS:     ${catalina_opts}"
-print "Found KMS_SSL_KEYSTORE_PASS:     `echo ${KMS_SSL_KEYSTORE_PASS} | sed 's/./*/g'`"
+# We're using hadoop-common, so set up some stuff it might need:
+hadoop_finalize
 
-export CATALINA_OPTS="${CATALINA_OPTS} ${catalina_opts}"
+hadoop_verify_logdir
+
+if [[ $# = 0 ]]; then
+  case "${HADOOP_DAEMON_MODE}" in
+    status)
+      hadoop_status_daemon "${CATALINA_PID}"
+      exit
+    ;;
+    start)
+      set -- "start"
+    ;;
+    stop)
+      set -- "stop"
+    ;;
+  esac
+fi
+
+hadoop_finalize_catalina_opts
+export CATALINA_OPTS
 
 # A bug in catalina.sh script does not use CATALINA_OPTS for stopping the server
 #
-if [ "${1}" = "stop" ]; then
+if [[ "${1}" = "stop" ]]; then
   export JAVA_OPTS=${CATALINA_OPTS}
 fi
 
 # If ssl, the populate the passwords into ssl-server.xml before starting tomcat
-if [ ! "${KMS_SSL_KEYSTORE_PASS}" = "" ] || [ ! "${KMS_SSL_TRUSTSTORE_PASS}" = "" ]; then
-  # Set a KEYSTORE_PASS if not already set
-  KMS_SSL_KEYSTORE_PASS=${KMS_SSL_KEYSTORE_PASS:-password}
-  cat ${CATALINA_BASE}/conf/ssl-server.xml.conf \
-    | sed 's/_kms_ssl_keystore_pass_/'${KMS_SSL_KEYSTORE_PASS}'/g' \
-    | sed 's/_kms_ssl_truststore_pass_/'${KMS_SSL_TRUSTSTORE_PASS}'/g' > ${CATALINA_BASE}/conf/ssl-server.xml
-fi 
+#
+# KMS_SSL_KEYSTORE_PASS is a bit odd.
+# if undefined, then the if test will not enable ssl on its own
+# if "", set it to "password".
+# if custom, use provided password
+#
+if [[ -f "${HADOOP_CATALINA_HOME}/conf/ssl-server.xml.conf" ]]; then
+  if [[ -n "${KMS_SSL_KEYSTORE_PASS+x}" ]] || [[ -n "${KMS_SSL_TRUSTSTORE_PASS}" ]]; then
+      export KMS_SSL_KEYSTORE_PASS=${KMS_SSL_KEYSTORE_PASS:-password}
+      sed -e 's/_kms_ssl_keystore_pass_/'${KMS_SSL_KEYSTORE_PASS}'/g' \
+          -e 's/_kms_ssl_truststore_pass_/'${KMS_SSL_TRUSTSTORE_PASS}'/g' \
+        "${HADOOP_CATALINA_HOME}/conf/ssl-server.xml.conf" \
+        > "${HADOOP_CATALINA_HOME}/conf/ssl-server.xml"
+      chmod 700 "${HADOOP_CATALINA_HOME}/conf/ssl-server.xml" >/dev/null 2>&1
+  fi
+fi
 
-exec ${KMS_CATALINA_HOME}/bin/catalina.sh "$@"
+exec "${HADOOP_CATALINA_HOME}/bin/catalina.sh" "$@"


[05/18] hadoop git commit: HADOOP-11039. ByteBufferReadable API doc is inconsistent with the implementations. (Yi Liu via Colin P. McCabe)

Posted by zh...@apache.org.
HADOOP-11039. ByteBufferReadable API doc is inconsistent with the implementations. (Yi Liu via Colin P. McCabe)


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

Branch: refs/heads/HDFS-EC
Commit: f123b71cd7d4bb2f135c6265b753cdbbcd2dacfd
Parents: 43dd72d
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Mon Dec 29 13:37:45 2014 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:36 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt          |  3 +++
 .../java/org/apache/hadoop/fs/ByteBufferReadable.java    | 11 +++++------
 2 files changed, 8 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f123b71c/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 4e25997..793e583 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -655,6 +655,9 @@ Release 2.7.0 - UNRELEASED
 
     HADOOP-11448. Fix findbugs warnings in FileBasedIPList. (ozawa)
 
+    HADOOP-11039. ByteBufferReadable API doc is inconsistent with the
+    implementations. (Yi Liu via Colin P. McCabe)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f123b71c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ByteBufferReadable.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ByteBufferReadable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ByteBufferReadable.java
index e18a115..aa6e85e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ByteBufferReadable.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ByteBufferReadable.java
@@ -29,9 +29,8 @@ public interface ByteBufferReadable {
    * Reads up to buf.remaining() bytes into buf. Callers should use
    * buf.limit(..) to control the size of the desired read.
    * <p/>
-   * After a successful call, buf.position() and buf.limit() should be
-   * unchanged, and therefore any data can be immediately read from buf.
-   * buf.mark() may be cleared or updated.
+   * After a successful call, buf.position() will be advanced by the number 
+   * of bytes read and buf.limit() should be unchanged.
    * <p/>
    * In the case of an exception, the values of buf.position() and buf.limit()
    * are undefined, and callers should be prepared to recover from this
@@ -45,9 +44,9 @@ public interface ByteBufferReadable {
    * signal an error upon their receipt.
    *
    * @param buf
-   *          the ByteBuffer to receive the results of the read operation. Up to
-   *          buf.limit() - buf.position() bytes may be read.
-   * @return the number of bytes available to read from buf
+   *          the ByteBuffer to receive the results of the read operation.
+   * @return the number of bytes read, possibly zero, or -1 if 
+   *         reach end-of-stream
    * @throws IOException
    *           if there is some error performing the read
    */


[17/18] hadoop git commit: HADOOP-11390 Metrics 2 ganglia provider to include hostname in unresolved address problems. (Varun Saxena via stevel)

Posted by zh...@apache.org.
HADOOP-11390 Metrics 2 ganglia provider to include hostname in unresolved address problems. (Varun Saxena via stevel)


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

Branch: refs/heads/HDFS-EC
Commit: f3aced275867384cf18be5c1e1b520e88c6aee5d
Parents: e9e00ac
Author: Steve Loughran <st...@apache.org>
Authored: Mon Jan 5 16:09:25 2015 +0000
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:38 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt               | 3 +++
 .../hadoop/metrics2/sink/ganglia/AbstractGangliaSink.java     | 7 +++++++
 2 files changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f3aced27/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index baf68d6..0c76894 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -455,6 +455,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11455. KMS and Credential CLI should request confirmation for
     deletion by default. (Charles Lamb via yliu)
 
+    HADOOP-11390 Metrics 2 ganglia provider to include hostname in
+    unresolved address problems. (Varun Saxena via stevel)
+
   OPTIMIZATIONS
 
     HADOOP-11323. WritableComparator#compare keeps reference to byte array.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f3aced27/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/AbstractGangliaSink.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/AbstractGangliaSink.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/AbstractGangliaSink.java
index 164ea08..88ad647 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/AbstractGangliaSink.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/AbstractGangliaSink.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.metrics2.sink.ganglia;
 import java.io.IOException;
 import java.net.DatagramPacket;
 import java.net.DatagramSocket;
+import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.net.SocketException;
 import java.net.UnknownHostException;
@@ -257,6 +258,12 @@ public abstract class AbstractGangliaSink implements MetricsSink {
   protected void emitToGangliaHosts() throws IOException {
     try {
       for (SocketAddress socketAddress : metricsServers) {
+        if (socketAddress == null || !(socketAddress instanceof InetSocketAddress))
+          throw new IllegalArgumentException("Unsupported Address type");
+        InetSocketAddress inetAddress = (InetSocketAddress)socketAddress;
+        if(inetAddress.isUnresolved()) {
+          throw new UnknownHostException("Unresolved host: " + inetAddress);
+        }
         DatagramPacket packet =
           new DatagramPacket(buffer, offset, socketAddress);
         datagramSocket.send(packet);


[06/18] hadoop git commit: HADOOP-11455. KMS and Credential CLI should request confirmation for deletion by default. (Charles Lamb via yliu)

Posted by zh...@apache.org.
HADOOP-11455. KMS and Credential CLI should request confirmation for deletion by default. (Charles Lamb via yliu)


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

Branch: refs/heads/HDFS-EC
Commit: d8b51172cbbcd0647cc9bd6e57c98e651036cdc9
Parents: 0c94460
Author: yliu <yl...@apache.org>
Authored: Mon Jan 5 06:55:08 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:37 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +++
 .../org/apache/hadoop/crypto/key/KeyShell.java  | 18 ++++++++++--------
 .../hadoop/security/alias/CredentialShell.java  | 20 +++++++++++---------
 .../apache/hadoop/crypto/key/TestKeyShell.java  |  3 ++-
 .../hadoop/security/alias/TestCredShell.java    |  7 ++++---
 5 files changed, 30 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8b51172/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index d7ebeac..ec75e8d 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -452,6 +452,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11399. Java Configuration file and .xml files should be
     automatically cross-compared (rchiang via rkanter)
 
+    HADOOP-11455. KMS and Credential CLI should request confirmation for
+    deletion by default. (Charles Lamb via yliu)
+
   OPTIMIZATIONS
 
     HADOOP-11323. WritableComparator#compare keeps reference to byte array.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8b51172/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java
index e0ca624..4c72d06 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java
@@ -47,7 +47,7 @@ public class KeyShell extends Configured implements Tool {
       "   [" + ListCommand.USAGE + "]\n";
   private static final String LIST_METADATA = "keyShell.list.metadata";
 
-  private boolean interactive = false;
+  private boolean interactive = true;
   private Command command = null;
 
   /** allows stdout to be captured if necessary */
@@ -169,8 +169,8 @@ public class KeyShell extends Configured implements Tool {
         getConf().set(KeyProviderFactory.KEY_PROVIDER_PATH, args[++i]);
       } else if ("-metadata".equals(args[i])) {
         getConf().setBoolean(LIST_METADATA, true);
-      } else if ("-i".equals(args[i]) || ("-interactive".equals(args[i]))) {
-        interactive = true;
+      } else if ("-f".equals(args[i]) || ("-force".equals(args[i]))) {
+        interactive = false;
       } else if ("-help".equals(args[i])) {
         printKeyShellUsage();
         return 1;
@@ -367,11 +367,13 @@ public class KeyShell extends Configured implements Tool {
   }
 
   private class DeleteCommand extends Command {
-    public static final String USAGE = "delete <keyname> [-provider <provider>] [-help]";
+    public static final String USAGE =
+        "delete <keyname> [-provider <provider>] [-f] [-help]";
     public static final String DESC =
         "The delete subcommand deletes all versions of the key\n" +
         "specified by the <keyname> argument from within the\n" +
-        "provider specified -provider.";
+        "provider specified -provider. The command asks for\n" +
+        "user confirmation unless -f is specified.";
 
     String keyName = null;
     boolean cont = true;
@@ -397,10 +399,10 @@ public class KeyShell extends Configured implements Tool {
         try {
           cont = ToolRunner
               .confirmPrompt("You are about to DELETE all versions of "
-                  + " key: " + keyName + " from KeyProvider "
-                  + provider + ". Continue?:");
+                  + " key " + keyName + " from KeyProvider "
+                  + provider + ". Continue? ");
           if (!cont) {
-            out.println("Nothing has been be deleted.");
+            out.println(keyName + " has not been deleted.");
           }
           return cont;
         } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8b51172/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java
index 6d9c6af..f397403 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java
@@ -44,7 +44,7 @@ public class CredentialShell extends Configured implements Tool {
       "   [" + DeleteCommand.USAGE + "]\n" +
       "   [" + ListCommand.USAGE + "]\n";
 
-  private boolean interactive = false;
+  private boolean interactive = true;
   private Command command = null;
 
   /** allows stdout to be captured if necessary */
@@ -116,8 +116,8 @@ public class CredentialShell extends Configured implements Tool {
         userSuppliedProvider = true;
         getConf().set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, 
             args[++i]);
-      } else if (args[i].equals("-i") || (args[i].equals("-interactive"))) {
-        interactive = true;
+      } else if (args[i].equals("-f") || (args[i].equals("-force"))) {
+        interactive = false;
       } else if (args[i].equals("-v") || (args[i].equals("-value"))) {
         value = args[++i];
       } else if (args[i].equals("-help")) {
@@ -236,11 +236,13 @@ public class CredentialShell extends Configured implements Tool {
   }
 
   private class DeleteCommand extends Command {
-    public static final String USAGE = "delete <alias> [-provider] [-help]";
+    public static final String USAGE =
+        "delete <alias> [-provider] [-f] [-help]";
     public static final String DESC =
-        "The delete subcommand deletes the credenital\n" +
+        "The delete subcommand deletes the credential\n" +
         "specified as the <alias> argument from within the provider\n" +
-        "indicated through the -provider argument";
+        "indicated through the -provider argument. The command asks for\n" +
+        "confirmation unless the -f option is specified.";
 
     String alias = null;
     boolean cont = true;
@@ -267,9 +269,9 @@ public class CredentialShell extends Configured implements Tool {
       if (interactive) {
         try {
           cont = ToolRunner
-              .confirmPrompt("You are about to DELETE the credential: " + 
+              .confirmPrompt("You are about to DELETE the credential " +
                   alias + " from CredentialProvider " + provider.toString() +
-                  ". Continue?:");
+                  ". Continue? ");
           if (!cont) {
             out.println("Nothing has been deleted.");
           }
@@ -293,7 +295,7 @@ public class CredentialShell extends Configured implements Tool {
           provider.flush();
           printProviderWritten();
         } catch (IOException e) {
-          out.println(alias + "has NOT been deleted.");
+          out.println(alias + " has NOT been deleted.");
           throw e;
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8b51172/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyShell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyShell.java
index 3407eb7..fe88718 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyShell.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyShell.java
@@ -75,7 +75,8 @@ public class TestKeyShell {
   private void deleteKey(KeyShell ks, String keyName) throws Exception {
     int rc;
     outContent.reset();
-    final String[] delArgs = {"delete", keyName, "-provider", jceksProvider};
+    final String[] delArgs =
+        {"delete", keyName, "-f", "-provider", jceksProvider};
     rc = ks.run(delArgs);
     assertEquals(0, rc);
     assertTrue(outContent.toString().contains(keyName + " has been " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8b51172/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java
index c890362..7ba4bc1 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java
@@ -47,6 +47,7 @@ public class TestCredShell {
     System.setOut(new PrintStream(outContent));
     System.setErr(new PrintStream(errContent));
     final Path jksPath = new Path(tmpDir.toString(), "keystore.jceks");
+    new File(jksPath.toString()).delete();
     jceksProvider = "jceks://file" + jksPath.toUri();
   }
   
@@ -71,7 +72,7 @@ public class TestCredShell {
     assertTrue(outContent.toString().contains("credential1"));
 
     outContent.reset();
-    String[] args4 = {"delete", "credential1", "-provider",
+    String[] args4 = {"delete", "credential1", "-f", "-provider",
         jceksProvider};
     rc = cs.run(args4);
     assertEquals(0, rc);
@@ -113,7 +114,7 @@ public class TestCredShell {
     assertTrue(outContent.toString().contains("WARNING: you are modifying a " +
     		"transient provider."));
 
-    String[] args2 = {"delete", "credential1", "-provider", "user:///"};
+    String[] args2 = {"delete", "credential1", "-f", "-provider", "user:///"};
     rc = cs.run(args2);
     assertEquals(outContent.toString(), 0, rc);
     assertTrue(outContent.toString().contains("credential1 has been successfully " +
@@ -167,7 +168,7 @@ public class TestCredShell {
     assertTrue(outContent.toString().contains("credential1 has been successfully " +
         "created."));
     
-    String[] args2 = {"delete", "credential1", "-provider",
+    String[] args2 = {"delete", "credential1", "-f", "-provider",
         jceksProvider};
     rc = shell.run(args2);
     assertEquals(0, rc);


[13/18] hadoop git commit: YARN-2958. Made RMStateStore not update the last sequence number when updating the delegation token. Contributed by Varun Saxena.

Posted by zh...@apache.org.
YARN-2958. Made RMStateStore not update the last sequence number when updating the delegation token. Contributed by Varun Saxena.


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

Branch: refs/heads/HDFS-EC
Commit: 35805a695d81ec01880d18ed4ddf24f219c99c34
Parents: 9a3a069
Author: Zhijie Shen <zj...@apache.org>
Authored: Mon Jan 5 13:33:07 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:38 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../recovery/FileSystemRMStateStore.java        | 53 +++++++++-----------
 .../recovery/LeveldbRMStateStore.java           | 45 +++++++++--------
 .../recovery/MemoryRMStateStore.java            | 32 +++++++-----
 .../recovery/NullRMStateStore.java              | 12 ++---
 .../resourcemanager/recovery/RMStateStore.java  | 40 +++++++--------
 .../recovery/RMStateStoreRMDTEvent.java         | 10 +---
 .../recovery/ZKRMStateStore.java                | 40 +++++++--------
 .../RMDelegationTokenSecretManager.java         | 15 ++----
 .../recovery/RMStateStoreTestBase.java          | 13 ++---
 .../recovery/TestZKRMStateStore.java            | 12 ++---
 11 files changed, 131 insertions(+), 144 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/35805a69/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 0d33b4a..c464275 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -311,6 +311,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2922. ConcurrentModificationException in CapacityScheduler's LeafQueue.
     (Rohith Sharmaks via ozawa)
 
+    YARN-2958. Made RMStateStore not update the last sequence number when updating the
+    delegation token. (Varun Saxena via zjshen)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35805a69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.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/recovery/FileSystemRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
index 51e3916..7783662 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
@@ -60,8 +60,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.AM
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.EpochPBImpl;
-import org.apache.hadoop.yarn.util.ConverterUtils;
-
 import com.google.common.annotations.VisibleForTesting;
 
 @Private
@@ -452,11 +450,10 @@ public class FileSystemRMStateStore extends RMStateStore {
   }
 
   @Override
-  public synchronized void storeRMDelegationTokenAndSequenceNumberState(
-      RMDelegationTokenIdentifier identifier, Long renewDate,
-      int latestSequenceNumber) throws Exception {
-    storeOrUpdateRMDelegationTokenAndSequenceNumberState(
-        identifier, renewDate,latestSequenceNumber, false);
+  public synchronized void storeRMDelegationTokenState(
+      RMDelegationTokenIdentifier identifier, Long renewDate)
+      throws Exception {
+    storeOrUpdateRMDelegationTokenState(identifier, renewDate, false);
   }
 
   @Override
@@ -469,16 +466,15 @@ public class FileSystemRMStateStore extends RMStateStore {
   }
 
   @Override
-  protected void updateRMDelegationTokenAndSequenceNumberInternal(
-      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate,
-      int latestSequenceNumber) throws Exception {
-    storeOrUpdateRMDelegationTokenAndSequenceNumberState(
-        rmDTIdentifier, renewDate,latestSequenceNumber, true);
+  protected void updateRMDelegationTokenState(
+      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate)
+      throws Exception {
+    storeOrUpdateRMDelegationTokenState(rmDTIdentifier, renewDate, true);
   }
 
-  private void storeOrUpdateRMDelegationTokenAndSequenceNumberState(
+  private void storeOrUpdateRMDelegationTokenState(
       RMDelegationTokenIdentifier identifier, Long renewDate,
-      int latestSequenceNumber, boolean isUpdate) throws Exception {
+      boolean isUpdate) throws Exception {
     Path nodeCreatePath =
         getNodePath(rmDTSecretManagerRoot,
           DELEGATION_TOKEN_PREFIX + identifier.getSequenceNumber());
@@ -490,23 +486,24 @@ public class FileSystemRMStateStore extends RMStateStore {
     } else {
       LOG.info("Storing RMDelegationToken_" + identifier.getSequenceNumber());
       writeFile(nodeCreatePath, identifierData.toByteArray());
-    }
 
-    // store sequence number
-    Path latestSequenceNumberPath = getNodePath(rmDTSecretManagerRoot,
-          DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX + latestSequenceNumber);
-    LOG.info("Storing " + DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX
-        + latestSequenceNumber);
-    if (dtSequenceNumberPath == null) {
-      if (!createFile(latestSequenceNumberPath)) {
-        throw new Exception("Failed to create " + latestSequenceNumberPath);
-      }
-    } else {
-      if (!renameFile(dtSequenceNumberPath, latestSequenceNumberPath)) {
-        throw new Exception("Failed to rename " + dtSequenceNumberPath);
+      // store sequence number
+      Path latestSequenceNumberPath = getNodePath(rmDTSecretManagerRoot,
+            DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX
+            + identifier.getSequenceNumber());
+      LOG.info("Storing " + DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX
+          + identifier.getSequenceNumber());
+      if (dtSequenceNumberPath == null) {
+        if (!createFile(latestSequenceNumberPath)) {
+          throw new Exception("Failed to create " + latestSequenceNumberPath);
+        }
+      } else {
+        if (!renameFile(dtSequenceNumberPath, latestSequenceNumberPath)) {
+          throw new Exception("Failed to rename " + dtSequenceNumberPath);
+        }
       }
+      dtSequenceNumberPath = latestSequenceNumberPath;
     }
-    dtSequenceNumberPath = latestSequenceNumberPath;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35805a69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.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/recovery/LeveldbRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
index 0f880c8..2c92714 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
@@ -544,31 +544,30 @@ public class LeveldbRMStateStore extends RMStateStore {
       throw new IOException(e);
     }
   }
-
-  @Override
-  protected void storeRMDelegationTokenAndSequenceNumberState(
-      RMDelegationTokenIdentifier tokenId, Long renewDate,
-      int latestSequenceNumber) throws IOException {
+  
+  private void storeOrUpdateRMDT(RMDelegationTokenIdentifier tokenId,
+      Long renewDate, boolean isUpdate) throws IOException {
     String tokenKey = getRMDTTokenNodeKey(tokenId);
     RMDelegationTokenIdentifierData tokenData =
         new RMDelegationTokenIdentifierData(tokenId, renewDate);
-    ByteArrayOutputStream bs = new ByteArrayOutputStream();
-    DataOutputStream ds = new DataOutputStream(bs);
-    try {
-      ds.writeInt(latestSequenceNumber);
-    } finally {
-      ds.close();
-    }
     if (LOG.isDebugEnabled()) {
       LOG.debug("Storing token to " + tokenKey);
-      LOG.debug("Storing " + latestSequenceNumber + " to "
-          + RM_DT_SEQUENCE_NUMBER_KEY);
     }
     try {
       WriteBatch batch = db.createWriteBatch();
       try {
         batch.put(bytes(tokenKey), tokenData.toByteArray());
-        batch.put(bytes(RM_DT_SEQUENCE_NUMBER_KEY), bs.toByteArray());
+        if(!isUpdate) {
+          ByteArrayOutputStream bs = new ByteArrayOutputStream();
+          try (DataOutputStream ds = new DataOutputStream(bs)) {
+            ds.writeInt(tokenId.getSequenceNumber());
+          }
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Storing " + tokenId.getSequenceNumber() + " to "
+                + RM_DT_SEQUENCE_NUMBER_KEY);   
+          }
+          batch.put(bytes(RM_DT_SEQUENCE_NUMBER_KEY), bs.toByteArray());
+        }
         db.write(batch);
       } finally {
         batch.close();
@@ -579,11 +578,17 @@ public class LeveldbRMStateStore extends RMStateStore {
   }
 
   @Override
-  protected void updateRMDelegationTokenAndSequenceNumberInternal(
-      RMDelegationTokenIdentifier tokenId, Long renewDate,
-      int latestSequenceNumber) throws IOException {
-    storeRMDelegationTokenAndSequenceNumberState(tokenId, renewDate,
-        latestSequenceNumber);
+  protected void storeRMDelegationTokenState(
+      RMDelegationTokenIdentifier tokenId, Long renewDate)
+      throws IOException {
+    storeOrUpdateRMDT(tokenId, renewDate, false);
+  }
+
+  @Override
+  protected void updateRMDelegationTokenState(
+      RMDelegationTokenIdentifier tokenId, Long renewDate)
+      throws IOException {
+    storeOrUpdateRMDT(tokenId, renewDate, true);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35805a69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.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/recovery/MemoryRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java
index 917fdc1..3646949 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java
@@ -149,23 +149,30 @@ public class MemoryRMStateStore extends RMStateStore {
     }
   }
 
-  @Override
-  public synchronized void storeRMDelegationTokenAndSequenceNumberState(
-      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate,
-      int latestSequenceNumber) throws Exception {
+  private void storeOrUpdateRMDT(RMDelegationTokenIdentifier rmDTIdentifier,
+      Long renewDate, boolean isUpdate) throws Exception {
     Map<RMDelegationTokenIdentifier, Long> rmDTState =
         state.rmSecretManagerState.getTokenState();
     if (rmDTState.containsKey(rmDTIdentifier)) {
       IOException e = new IOException("RMDelegationToken: " + rmDTIdentifier
-              + "is already stored.");
+          + "is already stored.");
       LOG.info("Error storing info for RMDelegationToken: " + rmDTIdentifier, e);
       throw e;
     }
     rmDTState.put(rmDTIdentifier, renewDate);
-    state.rmSecretManagerState.dtSequenceNumber = latestSequenceNumber;
+    if(!isUpdate) {
+      state.rmSecretManagerState.dtSequenceNumber = 
+          rmDTIdentifier.getSequenceNumber();
+    }
     LOG.info("Store RMDT with sequence number "
-        + rmDTIdentifier.getSequenceNumber()
-        + ". And the latest sequence number is " + latestSequenceNumber);
+             + rmDTIdentifier.getSequenceNumber());
+  }
+
+  @Override
+  public synchronized void storeRMDelegationTokenState(
+      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate)
+      throws Exception {
+    storeOrUpdateRMDT(rmDTIdentifier, renewDate, false);
   }
 
   @Override
@@ -179,12 +186,11 @@ public class MemoryRMStateStore extends RMStateStore {
   }
 
   @Override
-  protected void updateRMDelegationTokenAndSequenceNumberInternal(
-      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate,
-      int latestSequenceNumber) throws Exception {
+  protected void updateRMDelegationTokenState(
+      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate)
+      throws Exception {
     removeRMDelegationTokenState(rmDTIdentifier);
-    storeRMDelegationTokenAndSequenceNumberState(
-        rmDTIdentifier, renewDate, latestSequenceNumber);
+    storeOrUpdateRMDT(rmDTIdentifier, renewDate, true);
     LOG.info("Update RMDT with sequence number "
         + rmDTIdentifier.getSequenceNumber());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35805a69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/NullRMStateStore.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/recovery/NullRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/NullRMStateStore.java
index f80c497..d2c1e9d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/NullRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/NullRMStateStore.java
@@ -77,9 +77,9 @@ public class NullRMStateStore extends RMStateStore {
   }
 
   @Override
-  public void storeRMDelegationTokenAndSequenceNumberState(
-      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate,
-      int latestSequenceNumber) throws Exception {
+  public void storeRMDelegationTokenState(
+      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate)
+      throws Exception {
     // Do nothing
   }
 
@@ -90,9 +90,9 @@ public class NullRMStateStore extends RMStateStore {
   }
 
   @Override
-  protected void updateRMDelegationTokenAndSequenceNumberInternal(
-      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate,
-      int latestSequenceNumber) throws Exception {
+  protected void updateRMDelegationTokenState(
+      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate)
+      throws Exception {
     // Do nothing
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35805a69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.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/recovery/RMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
index 3966dc4..bccde53 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
@@ -296,9 +296,8 @@ public abstract class RMStateStore extends AbstractService {
       RMStateStoreRMDTEvent dtEvent = (RMStateStoreRMDTEvent) event;
       try {
         LOG.info("Storing RMDelegationToken and SequenceNumber");
-        store.storeRMDelegationTokenAndSequenceNumberState(
-            dtEvent.getRmDTIdentifier(), dtEvent.getRenewDate(),
-            dtEvent.getLatestSequenceNumber());
+        store.storeRMDelegationTokenState(
+            dtEvent.getRmDTIdentifier(), dtEvent.getRenewDate());
       } catch (Exception e) {
         LOG.error("Error While Storing RMDelegationToken and SequenceNumber ",
             e);
@@ -341,9 +340,8 @@ public abstract class RMStateStore extends AbstractService {
       RMStateStoreRMDTEvent dtEvent = (RMStateStoreRMDTEvent) event;
       try {
         LOG.info("Updating RMDelegationToken and SequenceNumber");
-        store.updateRMDelegationTokenAndSequenceNumberInternal(
-            dtEvent.getRmDTIdentifier(), dtEvent.getRenewDate(),
-            dtEvent.getLatestSequenceNumber());
+        store.updateRMDelegationTokenState(
+            dtEvent.getRmDTIdentifier(), dtEvent.getRenewDate());
       } catch (Exception e) {
         LOG.error("Error While Updating RMDelegationToken and SequenceNumber ",
             e);
@@ -672,11 +670,10 @@ public abstract class RMStateStore extends AbstractService {
    * RMDTSecretManager call this to store the state of a delegation token
    * and sequence number
    */
-  public void storeRMDelegationTokenAndSequenceNumber(
-      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate,
-      int latestSequenceNumber) {
+  public void storeRMDelegationToken(
+      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate) {
     handleStoreEvent(new RMStateStoreRMDTEvent(rmDTIdentifier, renewDate,
-        latestSequenceNumber, RMStateStoreEventType.STORE_DELEGATION_TOKEN));
+        RMStateStoreEventType.STORE_DELEGATION_TOKEN));
   }
 
   /**
@@ -684,17 +681,17 @@ public abstract class RMStateStore extends AbstractService {
    * Derived classes must implement this method to store the state of
    * RMDelegationToken and sequence number
    */
-  protected abstract void storeRMDelegationTokenAndSequenceNumberState(
-      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate,
-      int latestSequenceNumber) throws Exception;
+  protected abstract void storeRMDelegationTokenState(
+      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate)
+      throws Exception;
 
   /**
    * RMDTSecretManager call this to remove the state of a delegation token
    */
   public void removeRMDelegationToken(
-      RMDelegationTokenIdentifier rmDTIdentifier, int sequenceNumber) {
+      RMDelegationTokenIdentifier rmDTIdentifier) {
     handleStoreEvent(new RMStateStoreRMDTEvent(rmDTIdentifier, null,
-        sequenceNumber, RMStateStoreEventType.REMOVE_DELEGATION_TOKEN));
+        RMStateStoreEventType.REMOVE_DELEGATION_TOKEN));
   }
 
   /**
@@ -708,11 +705,10 @@ public abstract class RMStateStore extends AbstractService {
    * RMDTSecretManager call this to update the state of a delegation token
    * and sequence number
    */
-  public void updateRMDelegationTokenAndSequenceNumber(
-      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate,
-      int latestSequenceNumber) {
+  public void updateRMDelegationToken(
+      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate) {
     handleStoreEvent(new RMStateStoreRMDTEvent(rmDTIdentifier, renewDate,
-        latestSequenceNumber, RMStateStoreEventType.UPDATE_DELEGATION_TOKEN));
+        RMStateStoreEventType.UPDATE_DELEGATION_TOKEN));
   }
 
   /**
@@ -720,9 +716,9 @@ public abstract class RMStateStore extends AbstractService {
    * Derived classes must implement this method to update the state of
    * RMDelegationToken and sequence number
    */
-  protected abstract void updateRMDelegationTokenAndSequenceNumberInternal(
-      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate,
-      int latestSequenceNumber) throws Exception;
+  protected abstract void updateRMDelegationTokenState(
+      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate)
+      throws Exception;
 
   /**
    * RMDTSecretManager call this to store the state of a master key

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35805a69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreRMDTEvent.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/recovery/RMStateStoreRMDTEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreRMDTEvent.java
index 4cd4d2e..a3519ff 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreRMDTEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreRMDTEvent.java
@@ -23,18 +23,16 @@ import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 public class RMStateStoreRMDTEvent extends RMStateStoreEvent {
   private RMDelegationTokenIdentifier rmDTIdentifier;
   private Long renewDate;
-  private int latestSequenceNumber;
 
   public RMStateStoreRMDTEvent(RMStateStoreEventType type) {
     super(type);
   }
 
   public RMStateStoreRMDTEvent(RMDelegationTokenIdentifier rmDTIdentifier,
-      Long renewDate, int latestSequenceNumber, RMStateStoreEventType type) {
+      Long renewDate, RMStateStoreEventType type) {
     this(type);
     this.rmDTIdentifier = rmDTIdentifier;
     this.renewDate = renewDate;
-    this.latestSequenceNumber = latestSequenceNumber;
   }
 
   public RMDelegationTokenIdentifier getRmDTIdentifier() {
@@ -44,8 +42,4 @@ public class RMStateStoreRMDTEvent extends RMStateStoreEvent {
   public Long getRenewDate() {
     return renewDate;
   }
-
-  public int getLatestSequenceNumber() {
-    return latestSequenceNumber;
-  }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35805a69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.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/recovery/ZKRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
index 2babc82..f3da21e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
@@ -698,12 +698,11 @@ public class ZKRMStateStore extends RMStateStore {
   }
 
   @Override
-  protected synchronized void storeRMDelegationTokenAndSequenceNumberState(
-      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate,
-      int latestSequenceNumber) throws Exception {
+  protected synchronized void storeRMDelegationTokenState(
+      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate)
+      throws Exception {
     ArrayList<Op> opList = new ArrayList<Op>();
-    addStoreOrUpdateOps(
-        opList, rmDTIdentifier, renewDate, latestSequenceNumber, false);
+    addStoreOrUpdateOps(opList, rmDTIdentifier, renewDate, false);
     doMultiWithRetries(opList);
   }
 
@@ -727,29 +726,27 @@ public class ZKRMStateStore extends RMStateStore {
   }
 
   @Override
-  protected synchronized void updateRMDelegationTokenAndSequenceNumberInternal(
-      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate,
-      int latestSequenceNumber) throws Exception {
+  protected synchronized void updateRMDelegationTokenState(
+      RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate)
+      throws Exception {
     ArrayList<Op> opList = new ArrayList<Op>();
     String nodeRemovePath =
         getNodePath(delegationTokensRootPath, DELEGATION_TOKEN_PREFIX
             + rmDTIdentifier.getSequenceNumber());
     if (existsWithRetries(nodeRemovePath, true) == null) {
       // in case znode doesn't exist
-      addStoreOrUpdateOps(
-          opList, rmDTIdentifier, renewDate, latestSequenceNumber, false);
+      addStoreOrUpdateOps(opList, rmDTIdentifier, renewDate, false);
       LOG.debug("Attempted to update a non-existing znode " + nodeRemovePath);
     } else {
       // in case znode exists
-      addStoreOrUpdateOps(
-          opList, rmDTIdentifier, renewDate, latestSequenceNumber, true);
+      addStoreOrUpdateOps(opList, rmDTIdentifier, renewDate, true);
     }
     doMultiWithRetries(opList);
   }
 
   private void addStoreOrUpdateOps(ArrayList<Op> opList,
       RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate,
-      int latestSequenceNumber, boolean isUpdate) throws Exception {
+      boolean isUpdate) throws Exception {
     // store RM delegation token
     String nodeCreatePath =
         getNodePath(delegationTokensRootPath, DELEGATION_TOKEN_PREFIX
@@ -769,16 +766,15 @@ public class ZKRMStateStore extends RMStateStore {
       } else {
         opList.add(Op.create(nodeCreatePath, identifierData.toByteArray(), zkAcl,
             CreateMode.PERSISTENT));
+        // Update Sequence number only while storing DT
+        seqOut.writeInt(rmDTIdentifier.getSequenceNumber());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug((isUpdate ? "Storing " : "Updating ") +
+                    dtSequenceNumberPath + ". SequenceNumber: "
+                    + rmDTIdentifier.getSequenceNumber());
+        }
+        opList.add(Op.setData(dtSequenceNumberPath, seqOs.toByteArray(), -1));
       }
-
-
-     seqOut.writeInt(latestSequenceNumber);
-     if (LOG.isDebugEnabled()) {
-        LOG.debug((isUpdate ? "Storing " : "Updating ") + dtSequenceNumberPath +
-            ". SequenceNumber: " + latestSequenceNumber);
-      }
-
-     opList.add(Op.setData(dtSequenceNumberPath, seqOs.toByteArray(), -1));
     } finally {
       seqOs.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35805a69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMDelegationTokenSecretManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMDelegationTokenSecretManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMDelegationTokenSecretManager.java
index 90706ff..83defc5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMDelegationTokenSecretManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMDelegationTokenSecretManager.java
@@ -29,10 +29,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
-import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.DelegationTokenInformation;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@@ -109,8 +107,7 @@ public class RMDelegationTokenSecretManager extends
     try {
       LOG.info("storing RMDelegation token with sequence number: "
           + identifier.getSequenceNumber());
-      rmContext.getStateStore().storeRMDelegationTokenAndSequenceNumber(
-        identifier, renewDate, identifier.getSequenceNumber());
+      rmContext.getStateStore().storeRMDelegationToken(identifier, renewDate);
     } catch (Exception e) {
       LOG.error("Error in storing RMDelegationToken with sequence number: "
           + identifier.getSequenceNumber());
@@ -124,11 +121,10 @@ public class RMDelegationTokenSecretManager extends
     try {
       LOG.info("updating RMDelegation token with sequence number: "
           + id.getSequenceNumber());
-      rmContext.getStateStore().updateRMDelegationTokenAndSequenceNumber(id,
-        renewDate, id.getSequenceNumber());
+      rmContext.getStateStore().updateRMDelegationToken(id, renewDate);
     } catch (Exception e) {
-      LOG.error("Error in updating persisted RMDelegationToken with sequence number: "
-            + id.getSequenceNumber());
+      LOG.error("Error in updating persisted RMDelegationToken" +
+                " with sequence number: " + id.getSequenceNumber());
       ExitUtil.terminate(1, e);
     }
   }
@@ -139,8 +135,7 @@ public class RMDelegationTokenSecretManager extends
     try {
       LOG.info("removing RMDelegation token with sequence number: "
           + ident.getSequenceNumber());
-      rmContext.getStateStore().removeRMDelegationToken(ident,
-        delegationTokenSequenceNumber);
+      rmContext.getStateStore().removeRMDelegationToken(ident);
     } catch (Exception e) {
       LOG.error("Error in removing RMDelegationToken with sequence number: "
           + ident.getSequenceNumber());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35805a69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.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/recovery/RMStateStoreTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
index 82ecac0..b01969b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
@@ -411,16 +411,15 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
     RMDelegationTokenIdentifier dtId1 =
         new RMDelegationTokenIdentifier(new Text("owner1"),
           new Text("renewer1"), new Text("realuser1"));
+    int sequenceNumber = 1111;
+    dtId1.setSequenceNumber(sequenceNumber);
     byte[] tokenBeforeStore = dtId1.getBytes();
     Long renewDate1 = new Long(System.currentTimeMillis());
-    int sequenceNumber = 1111;
-    store.storeRMDelegationTokenAndSequenceNumber(dtId1, renewDate1,
-      sequenceNumber);
+    store.storeRMDelegationToken(dtId1, renewDate1);
     modifyRMDelegationTokenState();
     Map<RMDelegationTokenIdentifier, Long> token1 =
         new HashMap<RMDelegationTokenIdentifier, Long>();
     token1.put(dtId1, renewDate1);
-
     // store delegation key;
     DelegationKey key = new DelegationKey(1234, 4321 , "keyBytes".getBytes());
     HashSet<DelegationKey> keySet = new HashSet<DelegationKey>();
@@ -440,9 +439,7 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
 
     // update RM delegation token;
     renewDate1 = new Long(System.currentTimeMillis());
-    ++sequenceNumber;
-    store.updateRMDelegationTokenAndSequenceNumber(
-        dtId1, renewDate1, sequenceNumber);
+    store.updateRMDelegationToken(dtId1, renewDate1);
     token1.put(dtId1, renewDate1);
 
     RMDTSecretManagerState updateSecretManagerState =
@@ -463,7 +460,7 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
         noKeySecretManagerState.getDTSequenceNumber());
 
     // check to delete delegationToken
-    store.removeRMDelegationToken(dtId1, sequenceNumber);
+    store.removeRMDelegationToken(dtId1);
     RMDTSecretManagerState noKeyAndTokenSecretManagerState =
         store.loadState().getRMDTSecretManagerState();
     token1.clear();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35805a69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.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/recovery/TestZKRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
index 2043484..87df3d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
@@ -337,20 +337,18 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     RMDelegationTokenIdentifier dtId1 =
         new RMDelegationTokenIdentifier(new Text("owner1"),
             new Text("renewer1"), new Text("realuser1"));
-    Long renewDate1 = new Long(System.currentTimeMillis());
-    int sequenceNumber = 1111;
-    store.storeRMDelegationTokenAndSequenceNumber(dtId1, renewDate1,
-        sequenceNumber);
+    Long renewDate1 = new Long(System.currentTimeMillis()); 
+    dtId1.setSequenceNumber(1111);
+    store.storeRMDelegationToken(dtId1, renewDate1);
     assertEquals("RMStateStore should have been in fenced state", true,
         store.isFencedState());
 
-    store.updateRMDelegationTokenAndSequenceNumber(dtId1, renewDate1,
-        sequenceNumber);
+    store.updateRMDelegationToken(dtId1, renewDate1);
     assertEquals("RMStateStore should have been in fenced state", true,
         store.isFencedState());
 
     // remove delegation key;
-    store.removeRMDelegationToken(dtId1, sequenceNumber);
+    store.removeRMDelegationToken(dtId1);
     assertEquals("RMStateStore should have been in fenced state", true,
         store.isFencedState());
 


[18/18] hadoop git commit: Move YARN-2360 from 2.6 to 2.7 in CHANGES.txt

Posted by zh...@apache.org.
Move YARN-2360 from 2.6 to 2.7 in CHANGES.txt


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

Branch: refs/heads/HDFS-EC
Commit: cf023117983c6e7be95742af2bac1042fa539874
Parents: 35805a6
Author: Karthik Kambatla <ka...@apache.org>
Authored: Tue Jan 6 03:31:11 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:38 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf023117/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index c464275..63faa60 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -161,6 +161,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2998. Abstract out scheduler independent PlanFollower components. 
     (Anubhav Dhoot via kasha)
 
+    YARN-2360. Fair Scheduler: Display dynamic fair share for queues on the 
+    scheduler page. (Ashwin Shankar and Wei Yan via kasha)
+
 
   OPTIMIZATIONS
 
@@ -554,9 +557,6 @@ Release 2.6.0 - 2014-11-18
     YARN-2406. Move RM recovery related proto to
     yarn_server_resourcemanager_recovery.proto. (Tsuyoshi Ozawa via jianhe)
 
-    YARN-2360. Fair Scheduler: Display dynamic fair share for queues on the 
-    scheduler page. (Ashwin Shankar and Wei Yan via kasha)
-
     YARN-1506. Changed RMNode/SchedulerNode to update resource with event
     notification. (Junping Du via jianhe)
 


[02/18] hadoop git commit: Revert "YARN-2492(wrong jira number). Added node-labels page on RM web UI. Contributed by Wangda Tan"

Posted by zh...@apache.org.
Revert "YARN-2492(wrong jira number). Added node-labels page on RM web UI. Contributed by Wangda Tan"

This reverts commit 5f57b904f550515693d93a2959e663b0d0260696.


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

Branch: refs/heads/HDFS-EC
Commit: 181827a5116259bd583b5350b919e8af2bff6db0
Parents: 756b66b
Author: Jian He <ji...@apache.org>
Authored: Tue Dec 30 16:46:26 2014 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:36 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   2 -
 .../nodelabels/CommonNodeLabelsManager.java     |  21 ++-
 .../hadoop/yarn/nodelabels/NodeLabel.java       |  96 -------------
 .../hadoop/yarn/webapp/YarnWebParams.java       |   1 -
 .../nodelabels/RMNodeLabelsManager.java         |  36 ++---
 .../server/resourcemanager/webapp/NavBlock.java |   3 +-
 .../resourcemanager/webapp/NodeLabelsPage.java  |  91 ------------
 .../resourcemanager/webapp/NodesPage.java       | 139 +++++++++----------
 .../server/resourcemanager/webapp/RMWebApp.java |   1 -
 .../resourcemanager/webapp/RmController.java    |   6 +-
 .../yarn/server/resourcemanager/MockNodes.java  |  46 +-----
 .../TestWorkPreservingRMRestart.java            |   2 +-
 .../nodelabels/TestRMNodeLabelsManager.java     |  33 -----
 .../resourcemanager/webapp/TestNodesPage.java   |  45 ------
 14 files changed, 101 insertions(+), 421 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/181827a5/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 0ebf8a3..e707ea7 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -155,8 +155,6 @@ Release 2.7.0 - UNRELEASED
     YARN-2993. Several fixes (missing acl check, error log msg ...) and some 
     refinement in AdminService. (Yi Liu via junping_du)
 
-    YARN-2943. Added node-labels page on RM web UI. (Wangda Tan via jianhe)
-
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/181827a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
index e888cc5..070aa1f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
@@ -72,8 +72,8 @@ public class CommonNodeLabelsManager extends AbstractService {
 
   protected Dispatcher dispatcher;
 
-  protected ConcurrentMap<String, NodeLabel> labelCollections =
-      new ConcurrentHashMap<String, NodeLabel>();
+  protected ConcurrentMap<String, Label> labelCollections =
+      new ConcurrentHashMap<String, Label>();
   protected ConcurrentMap<String, Host> nodeCollections =
       new ConcurrentHashMap<String, Host>();
 
@@ -82,6 +82,19 @@ public class CommonNodeLabelsManager extends AbstractService {
 
   protected NodeLabelsStore store;
 
+  protected static class Label {
+    private Resource resource;
+
+    protected Label() {
+      this.resource = Resource.newInstance(0, 0);
+    }
+
+    public Resource getResource() {
+      return this.resource;
+    }
+
+  }
+
   /**
    * A <code>Host</code> can have multiple <code>Node</code>s 
    */
@@ -188,7 +201,7 @@ public class CommonNodeLabelsManager extends AbstractService {
   protected void serviceInit(Configuration conf) throws Exception {
     initNodeLabelStore(conf);
     
-    labelCollections.put(NO_LABEL, new NodeLabel(NO_LABEL));
+    labelCollections.put(NO_LABEL, new Label());
   }
 
   protected void initNodeLabelStore(Configuration conf) throws Exception {
@@ -258,7 +271,7 @@ public class CommonNodeLabelsManager extends AbstractService {
     for (String label : labels) {
       // shouldn't overwrite it to avoid changing the Label.resource
       if (this.labelCollections.get(label) == null) {
-        this.labelCollections.put(label, new NodeLabel(label));
+        this.labelCollections.put(label, new Label());
         newLabels.add(label);
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/181827a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabel.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabel.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabel.java
deleted file mode 100644
index 7668648..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabel.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.nodelabels;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.util.resource.Resources;
-
-public class NodeLabel implements Comparable<NodeLabel> {
-  private Resource resource;
-  private int numActiveNMs;
-  private String labelName;
-
-  public NodeLabel(String labelName) {
-    this(labelName, Resource.newInstance(0, 0), 0);
-  }
-  
-  protected NodeLabel(String labelName, Resource res, int activeNMs) {
-    this.labelName = labelName;
-    this.resource = res;
-    this.numActiveNMs = activeNMs;
-  }
-  
-  public void addNode(Resource nodeRes) {
-    Resources.addTo(resource, nodeRes);
-    numActiveNMs++;
-  }
-  
-  public void removeNode(Resource nodeRes) {
-    Resources.subtractFrom(resource, nodeRes);
-    numActiveNMs--;
-  }
-
-  public Resource getResource() {
-    return this.resource;
-  }
-
-  public int getNumActiveNMs() {
-    return numActiveNMs;
-  }
-  
-  public String getLabelName() {
-    return labelName;
-  }
-  
-  public NodeLabel getCopy() {
-    return new NodeLabel(labelName, resource, numActiveNMs);
-  }
-  
-  @Override
-  public int compareTo(NodeLabel o) {
-    // We should always put empty label entry first after sorting
-    if (labelName.isEmpty() != o.getLabelName().isEmpty()) {
-      if (labelName.isEmpty()) {
-        return -1;
-      }
-      return 1;
-    }
-    
-    return labelName.compareTo(o.getLabelName());
-  }
-  
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof NodeLabel) {
-      NodeLabel other = (NodeLabel) obj;
-      return Resources.equals(resource, other.getResource())
-          && StringUtils.equals(labelName, other.getLabelName())
-          && (other.getNumActiveNMs() == numActiveNMs); 
-    }
-    return false;
-  }
-  
-  @Override
-  public int hashCode() {
-    final int prime = 502357;
-    return (int) ((((long) labelName.hashCode() << 8)
-        + (resource.hashCode() << 4) + numActiveNMs) % prime);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/181827a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
index 62c3c7a..91d2a20 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
@@ -32,5 +32,4 @@ public interface YarnWebParams {
   String APP_STATE = "app.state";
   String QUEUE_NAME = "queue.name";
   String NODE_STATE = "node.state";
-  String NODE_LABEL = "node.label";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/181827a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
index 828d1bc..646441a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
@@ -19,12 +19,10 @@
 package org.apache.hadoop.yarn.server.resourcemanager.nodelabels;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -39,7 +37,6 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
-import org.apache.hadoop.yarn.nodelabels.NodeLabel;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeLabelsUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -363,8 +360,8 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         // no label in the past
         if (oldLabels.isEmpty()) {
           // update labels
-          NodeLabel label = labelCollections.get(NO_LABEL);
-          label.removeNode(oldNM.resource);
+          Label label = labelCollections.get(NO_LABEL);
+          Resources.subtractFrom(label.getResource(), oldNM.resource);
 
           // update queues, all queue can access this node
           for (Queue q : queueCollections.values()) {
@@ -373,11 +370,11 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         } else {
           // update labels
           for (String labelName : oldLabels) {
-            NodeLabel label = labelCollections.get(labelName);
+            Label label = labelCollections.get(labelName);
             if (null == label) {
               continue;
             }
-            label.removeNode(oldNM.resource);
+            Resources.subtractFrom(label.getResource(), oldNM.resource);
           }
 
           // update queues, only queue can access this node will be subtract
@@ -398,8 +395,8 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         // no label in the past
         if (newLabels.isEmpty()) {
           // update labels
-          NodeLabel label = labelCollections.get(NO_LABEL);
-          label.addNode(newNM.resource);
+          Label label = labelCollections.get(NO_LABEL);
+          Resources.addTo(label.getResource(), newNM.resource);
 
           // update queues, all queue can access this node
           for (Queue q : queueCollections.values()) {
@@ -408,8 +405,8 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         } else {
           // update labels
           for (String labelName : newLabels) {
-            NodeLabel label = labelCollections.get(labelName);
-            label.addNode(newNM.resource);
+            Label label = labelCollections.get(labelName);
+            Resources.addTo(label.getResource(), newNM.resource);
           }
 
           // update queues, only queue can access this node will be subtract
@@ -478,21 +475,4 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
   public void setRMContext(RMContext rmContext) {
     this.rmContext = rmContext;
   }
-
-  public List<NodeLabel> pullRMNodeLabelsInfo() {
-    try {
-      readLock.lock();
-      List<NodeLabel> infos = new ArrayList<NodeLabel>();
-
-      for (Entry<String, NodeLabel> entry : labelCollections.entrySet()) {
-        NodeLabel label = entry.getValue();
-        infos.add(label.getCopy());
-      }
-
-      Collections.sort(infos);
-      return infos;
-    } finally {
-      readLock.unlock();
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/181827a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.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/NavBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.java
index db00bb0..ce8fd9e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.java
@@ -33,8 +33,7 @@ public class NavBlock extends HtmlBlock {
         h3("Cluster").
         ul().
           li().a(url("cluster"), "About")._().
-          li().a(url("nodes"), "Nodes")._().
-          li().a(url("nodelabels"), "Node Labels")._();
+          li().a(url("nodes"), "Nodes")._();
     UL<LI<UL<DIV<Hamlet>>>> subAppsList = mainList.
           li().a(url("apps"), "Applications").
             ul();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/181827a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java
deleted file mode 100644
index 5e8c1ed..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java
+++ /dev/null
@@ -1,91 +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;
-
-import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID;
-
-import org.apache.hadoop.yarn.nodelabels.NodeLabel;
-import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
-import org.apache.hadoop.yarn.webapp.SubView;
-import org.apache.hadoop.yarn.webapp.YarnWebParams;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
-import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
-
-import com.google.inject.Inject;
-
-public class NodeLabelsPage extends RmView {
-  static class NodeLabelsBlock extends HtmlBlock {
-    final ResourceManager rm;
-
-    @Inject
-    NodeLabelsBlock(ResourceManager rm, ViewContext ctx) {
-      super(ctx);
-      this.rm = rm;
-    }
-
-    @Override
-    protected void render(Block html) {
-      TBODY<TABLE<Hamlet>> tbody = html.table("#nodelabels").
-          thead().
-          tr().
-          th(".name", "Label Name").
-          th(".numOfActiveNMs", "Num Of Active NMs").
-          th(".totalResource", "Total Resource").
-          _()._().
-          tbody();
-  
-      RMNodeLabelsManager nlm = rm.getRMContext().getNodeLabelManager();
-      for (NodeLabel info : nlm.pullRMNodeLabelsInfo()) {
-        TR<TBODY<TABLE<Hamlet>>> row =
-            tbody.tr().td(
-                info.getLabelName().isEmpty() ? "<NO_LABEL>" : info
-                    .getLabelName());
-        int nActiveNMs = info.getNumActiveNMs();
-        if (nActiveNMs > 0) {
-          row = row.td()
-          .a(url("nodes",
-              "?" + YarnWebParams.NODE_LABEL + "=" + info.getLabelName()),
-              String.valueOf(nActiveNMs))
-           ._();
-        } else {
-          row = row.td(String.valueOf(nActiveNMs));
-        }
-        row.td(info.getResource().toString())._();
-      }
-      tbody._()._();
-    }
-  }
-
-  @Override protected void preHead(Page.HTML<_> html) {
-    commonPreHead(html);
-    String title = "Node labels of the cluster";
-    setTitle(title);
-    set(DATATABLES_ID, "nodelabels");
-    setTableStyles(html, "nodelabels", ".healthStatus {width:10em}",
-                   ".healthReport {width:10em}");
-  }
-
-  @Override protected Class<? extends SubView> content() {
-    return NodeLabelsBlock.class;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/181827a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.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/NodesPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java
index f28a9a8..d3849ae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java
@@ -1,25 +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.
- */
+* 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;
 
-import static org.apache.hadoop.yarn.webapp.YarnWebParams.NODE_STATE;
-import static org.apache.hadoop.yarn.webapp.YarnWebParams.NODE_LABEL;
+import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebApp.NODE_STATE;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
@@ -29,9 +28,7 @@ import java.util.Collection;
 
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.NodeState;
-import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
@@ -63,20 +60,26 @@ class NodesPage extends RmView {
 
       ResourceScheduler sched = rm.getResourceScheduler();
       String type = $(NODE_STATE);
-      String labelFilter = $(NODE_LABEL, CommonNodeLabelsManager.ANY).trim();
-      TBODY<TABLE<Hamlet>> tbody =
-          html.table("#nodes").thead().tr().th(".nodelabels", "Node Labels")
-              .th(".rack", "Rack").th(".state", "Node State")
-              .th(".nodeaddress", "Node Address")
-              .th(".nodehttpaddress", "Node HTTP Address")
-              .th(".lastHealthUpdate", "Last health-update")
-              .th(".healthReport", "Health-report")
-              .th(".containers", "Containers").th(".mem", "Mem Used")
-              .th(".mem", "Mem Avail").th(".vcores", "VCores Used")
-              .th(".vcores", "VCores Avail")
-              .th(".nodeManagerVersion", "Version")._()._().tbody();
+      TBODY<TABLE<Hamlet>> tbody = html.table("#nodes").
+          thead().
+          tr().
+          th(".nodelabels", "Node Labels").
+          th(".rack", "Rack").
+          th(".state", "Node State").
+          th(".nodeaddress", "Node Address").
+          th(".nodehttpaddress", "Node HTTP Address").
+          th(".lastHealthUpdate", "Last health-update").
+          th(".healthReport", "Health-report").
+          th(".containers", "Containers").
+          th(".mem", "Mem Used").
+          th(".mem", "Mem Avail").
+          th(".vcores", "VCores Used").
+          th(".vcores", "VCores Avail").
+          th(".nodeManagerVersion", "Version").
+          _()._().
+          tbody();
       NodeState stateFilter = null;
-      if (type != null && !type.isEmpty()) {
+      if(type != null && !type.isEmpty()) {
         stateFilter = NodeState.valueOf(type.toUpperCase());
       }
       Collection<RMNode> rmNodes = this.rm.getRMContext().getRMNodes().values();
@@ -94,9 +97,9 @@ class NodesPage extends RmView {
         }
       }
       for (RMNode ni : rmNodes) {
-        if (stateFilter != null) {
+        if(stateFilter != null) {
           NodeState state = ni.getState();
-          if (!stateFilter.equals(state)) {
+          if(!stateFilter.equals(state)) {
             continue;
           }
         } else {
@@ -106,71 +109,61 @@ class NodesPage extends RmView {
             continue;
           }
         }
-        // Besides state, we need to filter label as well.
-        if (!labelFilter.equals(RMNodeLabelsManager.ANY)) {
-          if (labelFilter.isEmpty()) {
-            // Empty label filter means only shows nodes without label
-            if (!ni.getNodeLabels().isEmpty()) {
-              continue;
-            }
-          } else if (!ni.getNodeLabels().contains(labelFilter)) {
-            // Only nodes have given label can show on web page.
-            continue;
-          }
-        }
         NodeInfo info = new NodeInfo(ni, sched);
-        int usedMemory = (int) info.getUsedMemory();
-        int availableMemory = (int) info.getAvailableMemory();
-        TR<TBODY<TABLE<Hamlet>>> row =
-            tbody.tr().td(StringUtils.join(",", info.getNodeLabels()))
-                .td(info.getRack()).td(info.getState()).td(info.getNodeId());
+        int usedMemory = (int)info.getUsedMemory();
+        int availableMemory = (int)info.getAvailableMemory();
+        TR<TBODY<TABLE<Hamlet>>> row = tbody.tr().
+            td(StringUtils.join(",", info.getNodeLabels())).
+            td(info.getRack()).
+            td(info.getState()).
+            td(info.getNodeId());
         if (isInactive) {
           row.td()._("N/A")._();
         } else {
           String httpAddress = info.getNodeHTTPAddress();
-          row.td().a("//" + httpAddress, httpAddress)._();
+          row.td().a("//" + httpAddress,
+              httpAddress)._();
         }
-        row.td().br().$title(String.valueOf(info.getLastHealthUpdate()))._()
-            ._(Times.format(info.getLastHealthUpdate()))._()
-            .td(info.getHealthReport())
-            .td(String.valueOf(info.getNumContainers())).td().br()
-            .$title(String.valueOf(usedMemory))._()
-            ._(StringUtils.byteDesc(usedMemory * BYTES_IN_MB))._().td().br()
-            .$title(String.valueOf(availableMemory))._()
-            ._(StringUtils.byteDesc(availableMemory * BYTES_IN_MB))._()
-            .td(String.valueOf(info.getUsedVirtualCores()))
-            .td(String.valueOf(info.getAvailableVirtualCores()))
-            .td(ni.getNodeManagerVersion())._();
+        row.td().br().$title(String.valueOf(info.getLastHealthUpdate()))._().
+              _(Times.format(info.getLastHealthUpdate()))._().
+            td(info.getHealthReport()).
+            td(String.valueOf(info.getNumContainers())).
+            td().br().$title(String.valueOf(usedMemory))._().
+              _(StringUtils.byteDesc(usedMemory * BYTES_IN_MB))._().
+            td().br().$title(String.valueOf(availableMemory))._().
+              _(StringUtils.byteDesc(availableMemory * BYTES_IN_MB))._().
+              td(String.valueOf(info.getUsedVirtualCores())).
+              td(String.valueOf(info.getAvailableVirtualCores())).
+            td(ni.getNodeManagerVersion()).
+            _();
       }
       tbody._()._();
     }
   }
 
-  @Override
-  protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<_> html) {
     commonPreHead(html);
     String type = $(NODE_STATE);
     String title = "Nodes of the cluster";
-    if (type != null && !type.isEmpty()) {
-      title = title + " (" + type + ")";
+    if(type != null && !type.isEmpty()) {
+      title = title+" ("+type+")";
     }
     setTitle(title);
     set(DATATABLES_ID, "nodes");
     set(initID(DATATABLES, "nodes"), nodesTableInit());
     setTableStyles(html, "nodes", ".healthStatus {width:10em}",
-        ".healthReport {width:10em}");
+                   ".healthReport {width:10em}");
   }
 
-  @Override
-  protected Class<? extends SubView> content() {
+  @Override protected Class<? extends SubView> content() {
     return NodesBlock.class;
   }
 
   private String nodesTableInit() {
     StringBuilder b = tableInit().append(", aoColumnDefs: [");
     b.append("{'bSearchable': false, 'aTargets': [ 6 ]}");
-    b.append(", {'sType': 'title-numeric', 'bSearchable': false, "
-        + "'aTargets': [ 7, 8 ] }");
+    b.append(", {'sType': 'title-numeric', 'bSearchable': false, " +
+        "'aTargets': [ 7, 8 ] }");
     b.append(", {'sType': 'title-numeric', 'aTargets': [ 4 ]}");
     b.append("]}");
     return b.toString();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/181827a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.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/RMWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
index c0e6834..67c73b8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
@@ -61,7 +61,6 @@ public class RMWebApp extends WebApp implements YarnWebParams {
     route(pajoin("/app", APPLICATION_ID), RmController.class, "app");
     route("/scheduler", RmController.class, "scheduler");
     route(pajoin("/queue", QUEUE_NAME), RmController.class, "queue");
-    route("/nodelabels", RmController.class, "nodelabels");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/181827a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.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/RmController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
index 972432b..f186bf4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.util.StringHelper;
 import org.apache.hadoop.yarn.webapp.Controller;
+import org.apache.hadoop.yarn.webapp.WebAppException;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
 
 import com.google.inject.Inject;
@@ -92,9 +93,4 @@ public class RmController extends Controller {
   public void submit() {
     setTitle("Application Submission Not Allowed");
   }
-  
-  public void nodelabels() {
-    setTitle("Node Labels");
-    render(NodeLabelsPage.class);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/181827a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.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/MockNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
index 2d863d1..278c151 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
@@ -30,13 +30,11 @@ import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 
-import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
 /**
@@ -55,12 +53,7 @@ public class MockNodes {
           // One unhealthy node per rack.
           list.add(nodeInfo(i, perNode, NodeState.UNHEALTHY));
         }
-        if (j == 0) {
-          // One node with label
-          list.add(nodeInfo(i, perNode, NodeState.RUNNING, ImmutableSet.of("x")));
-        } else {
-          list.add(newNodeInfo(i, perNode));
-        }
+        list.add(newNodeInfo(i, perNode));
       }
     }
     return list;
@@ -107,12 +100,10 @@ public class MockNodes {
     private String healthReport;
     private long lastHealthReportTime;
     private NodeState state;
-    private Set<String> labels;
 
     public MockRMNodeImpl(NodeId nodeId, String nodeAddr, String httpAddress,
         Resource perNode, String rackName, String healthReport,
-        long lastHealthReportTime, int cmdPort, String hostName, NodeState state,
-        Set<String> labels) {
+        long lastHealthReportTime, int cmdPort, String hostName, NodeState state) {
       this.nodeId = nodeId;
       this.nodeAddr = nodeAddr;
       this.httpAddress = httpAddress;
@@ -123,7 +114,6 @@ public class MockNodes {
       this.cmdPort = cmdPort;
       this.hostName = hostName;
       this.state = state;
-      this.labels = labels;
     }
 
     @Override
@@ -217,33 +207,16 @@ public class MockNodes {
 
     @Override
     public Set<String> getNodeLabels() {
-      if (labels != null) {
-        return labels;
-      }
-      return CommonNodeLabelsManager.EMPTY_STRING_SET;
+      return RMNodeLabelsManager.EMPTY_STRING_SET;
     }
   };
 
-  private static RMNode buildRMNode(int rack, final Resource perNode,
-      NodeState state, String httpAddr) {
-    return buildRMNode(rack, perNode, state, httpAddr, null);
-  }
-  
-  private static RMNode buildRMNode(int rack, final Resource perNode,
-      NodeState state, String httpAddr, Set<String> labels) {
-    return buildRMNode(rack, perNode, state, httpAddr, NODE_ID++, null, 123,
-        labels);
-  }
-  
-  private static RMNode buildRMNode(int rack, final Resource perNode,
-      NodeState state, String httpAddr, int hostnum, String hostName, int port) {
-    return buildRMNode(rack, perNode, state, httpAddr, hostnum, hostName, port,
-        null);
+  private static RMNode buildRMNode(int rack, final Resource perNode, NodeState state, String httpAddr) {
+    return buildRMNode(rack, perNode, state, httpAddr, NODE_ID++, null, 123);
   }
 
   private static RMNode buildRMNode(int rack, final Resource perNode,
-      NodeState state, String httpAddr, int hostnum, String hostName, int port,
-      Set<String> labels) {
+      NodeState state, String httpAddr, int hostnum, String hostName, int port) {
     final String rackName = "rack"+ rack;
     final int nid = hostnum;
     final String nodeAddr = hostName + ":" + nid;
@@ -255,18 +228,13 @@ public class MockNodes {
     final String httpAddress = httpAddr;
     String healthReport = (state == NodeState.UNHEALTHY) ? null : "HealthyMe";
     return new MockRMNodeImpl(nodeID, nodeAddr, httpAddress, perNode,
-        rackName, healthReport, 0, nid, hostName, state, labels);
+        rackName, healthReport, 0, nid, hostName, state);
   }
 
   public static RMNode nodeInfo(int rack, final Resource perNode,
       NodeState state) {
     return buildRMNode(rack, perNode, state, "N/A");
   }
-  
-  public static RMNode nodeInfo(int rack, final Resource perNode,
-      NodeState state, Set<String> labels) {
-    return buildRMNode(rack, perNode, state, "N/A", labels);
-  }
 
   public static RMNode newNodeInfo(int rack, final Resource perNode) {
     return buildRMNode(rack, perNode, NodeState.RUNNING, "localhost:0");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/181827a5/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 e21fcf9..842eaec 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
@@ -839,7 +839,7 @@ public class TestWorkPreservingRMRestart {
   // Test if RM on recovery receives the container release request from AM
   // before it receives the container status reported by NM for recovery. this
   // container should not be recovered.
-  @Test (timeout = 50000)
+  @Test (timeout = 30000)
   public void testReleasedContainerNotRecovered() throws Exception {
     MemoryRMStateStore memStore = new MemoryRMStateStore();
     memStore.init(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/181827a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
index e4cdc71..ed675f3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.nodelabels;
 
 import java.io.IOException;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
@@ -28,7 +27,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
-import org.apache.hadoop.yarn.nodelabels.NodeLabel;
 import org.apache.hadoop.yarn.nodelabels.NodeLabelTestBase;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.After;
@@ -430,35 +428,4 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
       Assert.fail("IOException from removeLabelsFromNode " + e);
     }
   }
-  
-  private void checkNodeLabelInfo(List<NodeLabel> infos, String labelName, int activeNMs, int memory) {
-    for (NodeLabel info : infos) {
-      if (info.getLabelName().equals(labelName)) {
-        Assert.assertEquals(activeNMs, info.getNumActiveNMs());
-        Assert.assertEquals(memory, info.getResource().getMemory());
-        return;
-      }
-    }
-    Assert.fail("Failed to find info has label=" + labelName);
-  }
-  
-  @Test(timeout = 5000)
-  public void testPullRMNodeLabelsInfo() throws IOException {
-    mgr.addToCluserNodeLabels(toSet("x", "y", "z"));
-    mgr.activateNode(NodeId.newInstance("n1", 1), Resource.newInstance(10, 0));
-    mgr.activateNode(NodeId.newInstance("n2", 1), Resource.newInstance(10, 0));
-    mgr.activateNode(NodeId.newInstance("n3", 1), Resource.newInstance(10, 0));
-    mgr.activateNode(NodeId.newInstance("n4", 1), Resource.newInstance(10, 0));
-    mgr.activateNode(NodeId.newInstance("n5", 1), Resource.newInstance(10, 0));
-    mgr.replaceLabelsOnNode(ImmutableMap.of(toNodeId("n1"), toSet("x"),
-        toNodeId("n2"), toSet("x"), toNodeId("n3"), toSet("y")));
-    
-    // x, y, z and ""
-    List<NodeLabel> infos = mgr.pullRMNodeLabelsInfo();
-    Assert.assertEquals(4, infos.size());
-    checkNodeLabelInfo(infos, RMNodeLabelsManager.NO_LABEL, 2, 20);
-    checkNodeLabelInfo(infos, "x", 2, 20);
-    checkNodeLabelInfo(infos, "y", 1, 10);
-    checkNodeLabelInfo(infos, "z", 0, 0);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/181827a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.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/TestNodesPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java
index 62713cf..bb38079 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java
@@ -106,49 +106,4 @@ public class TestNodesPage {
             * numberOfActualTableHeaders + numberOfThInMetricsTable)).print(
         "<td");
   }
-  
-  @Test
-  public void testNodesBlockRenderForNodeLabelFilterWithNonEmptyLabel() {
-    NodesBlock nodesBlock = injector.getInstance(NodesBlock.class);
-    nodesBlock.set("node.label", "x");
-    nodesBlock.render();
-    PrintWriter writer = injector.getInstance(PrintWriter.class);
-    WebAppTests.flushOutput(injector);
-
-    Mockito.verify(
-        writer,
-        Mockito.times(numberOfRacks
-            * numberOfActualTableHeaders + numberOfThInMetricsTable)).print(
-        "<td");
-  }
-  
-  @Test
-  public void testNodesBlockRenderForNodeLabelFilterWithEmptyLabel() {
-    NodesBlock nodesBlock = injector.getInstance(NodesBlock.class);
-    nodesBlock.set("node.label", "");
-    nodesBlock.render();
-    PrintWriter writer = injector.getInstance(PrintWriter.class);
-    WebAppTests.flushOutput(injector);
-
-    Mockito.verify(
-        writer,
-        Mockito.times(numberOfRacks * (numberOfNodesPerRack - 1)
-            * numberOfActualTableHeaders + numberOfThInMetricsTable)).print(
-        "<td");
-  }
-  
-  @Test
-  public void testNodesBlockRenderForNodeLabelFilterWithAnyLabel() {
-    NodesBlock nodesBlock = injector.getInstance(NodesBlock.class);
-    nodesBlock.set("node.label", "*");
-    nodesBlock.render();
-    PrintWriter writer = injector.getInstance(PrintWriter.class);
-    WebAppTests.flushOutput(injector);
-
-    Mockito.verify(
-        writer,
-        Mockito.times(numberOfRacks * numberOfNodesPerRack
-            * numberOfActualTableHeaders + numberOfThInMetricsTable)).print(
-        "<td");
-  }
 }


[08/18] hadoop git commit: YARN-2987. Fixed ClientRMService#getQueueInfo to check against queue and app ACLs. Contributed by Varun Saxena

Posted by zh...@apache.org.
YARN-2987. Fixed ClientRMService#getQueueInfo to check against queue and app ACLs. Contributed by Varun Saxena


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

Branch: refs/heads/HDFS-EC
Commit: 82add253a5959c5321a587f4c1fc966186589fbd
Parents: d5a33b4
Author: Jian He <ji...@apache.org>
Authored: Tue Dec 30 17:15:37 2014 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:37 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../server/resourcemanager/ClientRMService.java | 19 +++++++++++-
 .../resourcemanager/TestClientRMService.java    | 31 +++++++++++++++++++-
 3 files changed, 51 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/82add253/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 0ebf8a3..f2115e1 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -297,6 +297,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2938. Fixed new findbugs warnings in hadoop-yarn-resourcemanager and
     hadoop-yarn-applicationhistoryservice. (Varun Saxena via zjshen)
 
+    YARN-2987. Fixed ClientRMService#getQueueInfo to check against queue and
+    app ACLs. (Varun Saxena via jianhe)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82add253/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.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/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
index bee6bf8..0c37eb9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
@@ -827,6 +827,14 @@ public class ClientRMService extends AbstractService implements
   @Override
   public GetQueueInfoResponse getQueueInfo(GetQueueInfoRequest request)
       throws YarnException {
+    UserGroupInformation callerUGI;
+    try {
+      callerUGI = UserGroupInformation.getCurrentUser();
+    } catch (IOException ie) {
+      LOG.info("Error getting UGI ", ie);
+      throw RPCUtil.getRemoteException(ie);
+    }
+
     GetQueueInfoResponse response =
       recordFactory.newRecordInstance(GetQueueInfoResponse.class);
     try {
@@ -841,7 +849,16 @@ public class ClientRMService extends AbstractService implements
         appReports = new ArrayList<ApplicationReport>(apps.size());
         for (ApplicationAttemptId app : apps) {
           RMApp rmApp = rmContext.getRMApps().get(app.getApplicationId());
-          appReports.add(rmApp.createAndGetApplicationReport(null, true));
+          if (rmApp != null) {
+            // Check if user is allowed access to this app
+            if (!checkAccess(callerUGI, rmApp.getUser(),
+                ApplicationAccessType.VIEW_APP, rmApp)) {
+              continue;
+            }
+            appReports.add(
+                rmApp.createAndGetApplicationReport(
+                    callerUGI.getUserName(), true));
+          }          
         }
       }
       queueInfo.setApplications(appReports);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82add253/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.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/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
index a344e9a..a684346 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
@@ -553,8 +553,17 @@ public class TestClientRMService {
     YarnScheduler yarnScheduler = mock(YarnScheduler.class);
     RMContext rmContext = mock(RMContext.class);
     mockRMContext(yarnScheduler, rmContext);
+
+    ApplicationACLsManager mockAclsManager = mock(ApplicationACLsManager.class);
+    QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.class);
+    when(mockQueueACLsManager.checkAccess(any(UserGroupInformation.class),
+        any(QueueACL.class), anyString())).thenReturn(true);
+    when(mockAclsManager.checkAccess(any(UserGroupInformation.class),
+        any(ApplicationAccessType.class), anyString(),
+        any(ApplicationId.class))).thenReturn(true);
+
     ClientRMService rmService = new ClientRMService(rmContext, yarnScheduler,
-        null, null, null, null);
+        null, mockAclsManager, mockQueueACLsManager, null);
     GetQueueInfoRequest request = recordFactory
         .newRecordInstance(GetQueueInfoRequest.class);
     request.setQueueName("testqueue");
@@ -567,6 +576,26 @@ public class TestClientRMService {
     request.setIncludeApplications(true);
     // should not throw exception on nonexistent queue
     queueInfo = rmService.getQueueInfo(request);
+
+    // Case where user does not have application access
+    ApplicationACLsManager mockAclsManager1 =
+        mock(ApplicationACLsManager.class);
+    QueueACLsManager mockQueueACLsManager1 =
+        mock(QueueACLsManager.class);
+    when(mockQueueACLsManager1.checkAccess(any(UserGroupInformation.class),
+        any(QueueACL.class), anyString())).thenReturn(false);
+    when(mockAclsManager1.checkAccess(any(UserGroupInformation.class),
+        any(ApplicationAccessType.class), anyString(),
+        any(ApplicationId.class))).thenReturn(false);
+
+    ClientRMService rmService1 = new ClientRMService(rmContext, yarnScheduler,
+        null, mockAclsManager1, mockQueueACLsManager1, null);
+    request.setQueueName("testqueue");
+    request.setIncludeApplications(true);
+    GetQueueInfoResponse queueInfo1 = rmService1.getQueueInfo(request);
+    List<ApplicationReport> applications1 = queueInfo1.getQueueInfo()
+        .getApplications();
+    Assert.assertEquals(0, applications1.size());
   }
 
   private static final UserGroupInformation owner =


[16/18] hadoop git commit: HDFS-7572. TestLazyPersistFiles#testDnRestartWithSavedReplicas is flaky on Windows. Contributed by Arpit Agarwal.

Posted by zh...@apache.org.
HDFS-7572. TestLazyPersistFiles#testDnRestartWithSavedReplicas is flaky on Windows. Contributed by Arpit Agarwal.


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

Branch: refs/heads/HDFS-EC
Commit: 9a3a0697c2f6acac8de5fe3c85a6d7bddcc58c95
Parents: f3aced2
Author: cnauroth <cn...@apache.org>
Authored: Mon Jan 5 10:39:15 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:38 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java | 1 +
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a3a0697/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 8117383..48e3695 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -639,6 +639,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7563. NFS gateway parseStaticMap NumberFormatException 
     (Yongjun Zhang via brandonli)
 
+    HDFS-7572. TestLazyPersistFiles#testDnRestartWithSavedReplicas is flaky on
+    Windows. (Arpit Agarwal via cnauroth)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a3a0697/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
index 49d3c6d..ad2197a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
@@ -633,6 +633,7 @@ public class TestLazyPersistFiles extends LazyPersistTestCase {
     LOG.info("Restarting the DataNode");
     cluster.restartDataNode(0, true);
     cluster.waitActive();
+    triggerBlockReport();
 
     // Ensure that the replica is now on persistent storage.
     ensureFileReplicasOnStorageType(path1, DEFAULT);


[15/18] hadoop git commit: HADOOP-10908. Common needs updates for shell rewrite (aw)

Posted by zh...@apache.org.
HADOOP-10908. Common needs updates for shell rewrite (aw)


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

Branch: refs/heads/HDFS-EC
Commit: e38cd055b3441d5c6c1bfbbc1a312c08fdf5f25b
Parents: cf02311
Author: Allen Wittenauer <aw...@apache.org>
Authored: Mon Jan 5 14:26:41 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:38 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   2 +
 .../src/site/apt/ClusterSetup.apt.vm            | 348 ++++++++-----------
 .../src/site/apt/CommandsManual.apt.vm          | 316 +++++++++--------
 .../src/site/apt/FileSystemShell.apt.vm         | 313 ++++++++++-------
 .../src/site/apt/SingleCluster.apt.vm           |  20 +-
 5 files changed, 534 insertions(+), 465 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e38cd055/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 0c76894..40e8d29 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -344,6 +344,8 @@ Trunk (Unreleased)
 
     HADOOP-11397. Can't override HADOOP_IDENT_STRING (Kengo Seki via aw)
 
+    HADOOP-10908. Common needs updates for shell rewrite (aw)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e38cd055/hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm b/hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm
index f5f1deb..52b0552 100644
--- a/hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm
+++ b/hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm
@@ -11,83 +11,81 @@
 ~~ limitations under the License. See accompanying LICENSE file.
 
   ---
-  Hadoop Map Reduce Next Generation-${project.version} - Cluster Setup
+  Hadoop ${project.version} - Cluster Setup
   ---
   ---
   ${maven.build.timestamp}
 
 %{toc|section=1|fromDepth=0}
 
-Hadoop MapReduce Next Generation - Cluster Setup
+Hadoop Cluster Setup
 
 * {Purpose}
 
-  This document describes how to install, configure and manage non-trivial
+  This document describes how to install and configure
   Hadoop clusters ranging from a few nodes to extremely large clusters
-  with thousands of nodes.
+  with thousands of nodes.  To play with Hadoop, you may first want to
+  install it on a single machine (see {{{./SingleCluster.html}Single Node Setup}}).
 
-  To play with Hadoop, you may first want to install it on a single
-  machine (see {{{./SingleCluster.html}Single Node Setup}}).
+  This document does not cover advanced topics such as {{{./SecureMode.html}Security}} or
+  High Availability.
 
 * {Prerequisites}
 
-  Download a stable version of Hadoop from Apache mirrors.
+  * Install Java. See the {{{http://wiki.apache.org/hadoop/HadoopJavaVersions}Hadoop Wiki}} for known good versions.
+  * Download a stable version of Hadoop from Apache mirrors.
 
 * {Installation}
 
   Installing a Hadoop cluster typically involves unpacking the software on all
-  the machines in the cluster or installing RPMs.
+  the machines in the cluster or installing it via a packaging system as
+  appropriate for your operating system.  It is important to divide up the hardware
+  into functions.
 
   Typically one machine in the cluster is designated as the NameNode and
-  another machine the as ResourceManager, exclusively. These are the masters.
+  another machine the as ResourceManager, exclusively. These are the masters. Other
+  services (such as Web App Proxy Server and MapReduce Job History server) are usually
+  run either on dedicated hardware or on shared infrastrucutre, depending upon the load.
 
   The rest of the machines in the cluster act as both DataNode and NodeManager.
   These are the slaves.
 
-* {Running Hadoop in Non-Secure Mode}
+* {Configuring Hadoop in Non-Secure Mode}
 
-  The following sections describe how to configure a Hadoop cluster.
-
-  {Configuration Files}
-
-    Hadoop configuration is driven by two types of important configuration files:
+    Hadoop's Java configuration is driven by two types of important configuration files:
 
       * Read-only default configuration - <<<core-default.xml>>>,
         <<<hdfs-default.xml>>>, <<<yarn-default.xml>>> and
         <<<mapred-default.xml>>>.
 
-      * Site-specific configuration - <<conf/core-site.xml>>,
-        <<conf/hdfs-site.xml>>, <<conf/yarn-site.xml>> and
-        <<conf/mapred-site.xml>>.
-
+      * Site-specific configuration - <<<etc/hadoop/core-site.xml>>>,
+        <<<etc/hadoop/hdfs-site.xml>>>, <<<etc/hadoop/yarn-site.xml>>> and
+        <<<etc/hadoop/mapred-site.xml>>>.
 
-    Additionally, you can control the Hadoop scripts found in the bin/
-    directory of the distribution, by setting site-specific values via the
-    <<conf/hadoop-env.sh>> and <<yarn-env.sh>>.
 
-  {Site Configuration}
+  Additionally, you can control the Hadoop scripts found in the bin/
+  directory of the distribution, by setting site-specific values via the
+  <<<etc/hadoop/hadoop-env.sh>>> and <<<etc/hadoop/yarn-env.sh>>>.
 
   To configure the Hadoop cluster you will need to configure the
   <<<environment>>> in which the Hadoop daemons execute as well as the
   <<<configuration parameters>>> for the Hadoop daemons.
 
-  The Hadoop daemons are NameNode/DataNode and ResourceManager/NodeManager.
+  HDFS daemons are NameNode, SecondaryNameNode, and DataNode.  YARN damones
+  are ResourceManager, NodeManager, and WebAppProxy.  If MapReduce is to be
+  used, then the MapReduce Job History Server will also be running.  For
+  large installations, these are generally running on separate hosts.
 
 
 ** {Configuring Environment of Hadoop Daemons}
 
-  Administrators should use the <<conf/hadoop-env.sh>> and
-  <<conf/yarn-env.sh>> script to do site-specific customization of the
-  Hadoop daemons' process environment.
+  Administrators should use the <<<etc/hadoop/hadoop-env.sh>>> and optionally the
+  <<<etc/hadoop/mapred-env.sh>>> and <<<etc/hadoop/yarn-env.sh>>> scripts to do
+  site-specific customization of the Hadoop daemons' process environment.
 
-  At the very least you should specify the <<<JAVA_HOME>>> so that it is
+  At the very least, you must specify the <<<JAVA_HOME>>> so that it is
   correctly defined on each remote node.
 
-  In most cases you should also specify <<<HADOOP_PID_DIR>>> and
-  <<<HADOOP_SECURE_DN_PID_DIR>>> to point to directories that can only be
-  written to by the users that are going to run the hadoop daemons.
-  Otherwise there is the potential for a symlink attack.
-
   Administrators can configure individual daemons using the configuration
   options shown below in the table:
 
@@ -114,20 +112,42 @@ Hadoop MapReduce Next Generation - Cluster Setup
   statement should be added in hadoop-env.sh :
 
 ----
-  export HADOOP_NAMENODE_OPTS="-XX:+UseParallelGC ${HADOOP_NAMENODE_OPTS}"
+  export HADOOP_NAMENODE_OPTS="-XX:+UseParallelGC"
 ----
 
+  See <<<etc/hadoop/hadoop-env.sh>>> for other examples.
+
   Other useful configuration parameters that you can customize include:
 
-    * <<<HADOOP_LOG_DIR>>> / <<<YARN_LOG_DIR>>> - The directory where the
-      daemons' log files are stored. They are automatically created if they
-      don't exist.
+    * <<<HADOOP_PID_DIR>>> - The directory where the
+      daemons' process id files are stored.
+
+    * <<<HADOOP_LOG_DIR>>> - The directory where the
+      daemons' log files are stored. Log files are automatically created
+      if they don't exist.
+
+    * <<<HADOOP_HEAPSIZE_MAX>>> - The maximum amount of
+      memory to use for the Java heapsize.  Units supported by the JVM
+      are also supported here.  If no unit is present, it will be assumed
+      the number is in megabytes. By default, Hadoop will let the JVM
+      determine how much to use. This value can be overriden on
+      a per-daemon basis using the appropriate <<<_OPTS>>> variable listed above.
+      For example, setting <<<HADOOP_HEAPSIZE_MAX=1g>>> and
+      <<<HADOOP_NAMENODE_OPTS="-Xmx5g">>>  will configure the NameNode with 5GB heap.
+
+  In most cases, you should specify the <<<HADOOP_PID_DIR>>> and
+  <<<HADOOP_LOG_DIR>>> directories such that they can only be
+  written to by the users that are going to run the hadoop daemons.
+  Otherwise there is the potential for a symlink attack.
+
+  It is also traditional to configure <<<HADOOP_PREFIX>>> in the system-wide
+  shell environment configuration.  For example, a simple script inside
+  <<</etc/profile.d>>>:
 
-    * <<<HADOOP_HEAPSIZE>>> / <<<YARN_HEAPSIZE>>> - The maximum amount of
-      heapsize to use, in MB e.g. if the varibale is set to 1000 the heap
-      will be set to 1000MB.  This is used to configure the heap
-      size for the daemon. By default, the value is 1000.  If you want to
-      configure the values separately for each deamon you can use.
+---
+  HADOOP_PREFIX=/path/to/hadoop
+  export HADOOP_PREFIX
+---
 
 *--------------------------------------+--------------------------------------+
 || Daemon                              || Environment Variable                |
@@ -141,12 +161,12 @@ Hadoop MapReduce Next Generation - Cluster Setup
 | Map Reduce Job History Server        | HADOOP_JOB_HISTORYSERVER_HEAPSIZE    |
 *--------------------------------------+--------------------------------------+
 
-** {Configuring the Hadoop Daemons in Non-Secure Mode}
+** {Configuring the Hadoop Daemons}
 
     This section deals with important parameters to be specified in
     the given configuration files:
 
-    * <<<conf/core-site.xml>>>
+    * <<<etc/hadoop/core-site.xml>>>
 
 *-------------------------+-------------------------+------------------------+
 || Parameter              || Value                  || Notes                 |
@@ -157,7 +177,7 @@ Hadoop MapReduce Next Generation - Cluster Setup
 | | | Size of read/write buffer used in SequenceFiles. |
 *-------------------------+-------------------------+------------------------+
 
-    * <<<conf/hdfs-site.xml>>>
+    * <<<etc/hadoop/hdfs-site.xml>>>
 
       * Configurations for NameNode:
 
@@ -195,7 +215,7 @@ Hadoop MapReduce Next Generation - Cluster Setup
 | | | stored in all named directories, typically on different devices. |
 *-------------------------+-------------------------+------------------------+
 
-    * <<<conf/yarn-site.xml>>>
+    * <<<etc/hadoop/yarn-site.xml>>>
 
       * Configurations for ResourceManager and NodeManager:
 
@@ -341,9 +361,7 @@ Hadoop MapReduce Next Generation - Cluster Setup
 | | | Be careful, set this too small and you will spam the name node. |
 *-------------------------+-------------------------+------------------------+
 
-
-
-    * <<<conf/mapred-site.xml>>>
+    * <<<etc/hadoop/mapred-site.xml>>>
 
       * Configurations for MapReduce Applications:
 
@@ -395,22 +413,6 @@ Hadoop MapReduce Next Generation - Cluster Setup
 | | | Directory where history files are managed by the MR JobHistory Server. |
 *-------------------------+-------------------------+------------------------+
 
-* {Hadoop Rack Awareness}
-
-    The HDFS and the YARN components are rack-aware.
-
-    The NameNode and the ResourceManager obtains the rack information of the
-    slaves in the cluster by invoking an API <resolve> in an administrator
-    configured module.
-
-    The API resolves the DNS name (also IP address) to a rack id.
-
-    The site-specific module to use can be configured using the configuration
-    item <<<topology.node.switch.mapping.impl>>>. The default implementation
-    of the same runs a script/command configured using
-    <<<topology.script.file.name>>>. If <<<topology.script.file.name>>> is
-    not set, the rack id </default-rack> is returned for any passed IP address.
-
 * {Monitoring Health of NodeManagers}
 
     Hadoop provides a mechanism by which administrators can configure the
@@ -433,7 +435,7 @@ Hadoop MapReduce Next Generation - Cluster Setup
     node was healthy is also displayed on the web interface.
 
     The following parameters can be used to control the node health
-    monitoring script in <<<conf/yarn-site.xml>>>.
+    monitoring script in <<<etc/hadoop/yarn-site.xml>>>.
 
 *-------------------------+-------------------------+------------------------+
 || Parameter              || Value                  || Notes                 |
@@ -465,224 +467,170 @@ Hadoop MapReduce Next Generation - Cluster Setup
   disk is either raided or a failure in the boot disk is identified by the
   health checker script.
 
-* {Slaves file}
+* {Slaves File}
 
-  Typically you choose one machine in the cluster to act as the NameNode and
-  one machine as to act as the ResourceManager, exclusively. The rest of the
-  machines act as both a DataNode and NodeManager and are referred to as
-  <slaves>.
+  List all slave hostnames or IP addresses in your <<<etc/hadoop/slaves>>>
+  file, one per line.  Helper scripts (described below) will use the
+  <<<etc/hadoop/slaves>>> file to run commands on many hosts at once.  It is not
+  used for any of the Java-based Hadoop configuration.  In order
+  to use this functionality, ssh trusts (via either passphraseless ssh or
+  some other means, such as Kerberos) must be established for the accounts
+  used to run Hadoop.
 
-  List all slave hostnames or IP addresses in your <<<conf/slaves>>> file,
-  one per line.
+* {Hadoop Rack Awareness}
+
+  Many Hadoop components are rack-aware and take advantage of the
+  network topology for performance and safety. Hadoop daemons obtain the
+  rack information of the slaves in the cluster by invoking an administrator
+  configured module.  See the {{{./RackAwareness.html}Rack Awareness}}
+  documentation for more specific information.
+
+  It is highly recommended configuring rack awareness prior to starting HDFS.
 
 * {Logging}
 
-  Hadoop uses the Apache log4j via the Apache Commons Logging framework for
-  logging. Edit the <<<conf/log4j.properties>>> file to customize the
+  Hadoop uses the {{{http://logging.apache.org/log4j/2.x/}Apache log4j}} via the Apache Commons Logging framework for
+  logging. Edit the <<<etc/hadoop/log4j.properties>>> file to customize the
   Hadoop daemons' logging configuration (log-formats and so on).
 
 * {Operating the Hadoop Cluster}
 
   Once all the necessary configuration is complete, distribute the files to the
-  <<<HADOOP_CONF_DIR>>> directory on all the machines.
+  <<<HADOOP_CONF_DIR>>> directory on all the machines.  This should be the
+  same directory on all machines.
+
+  In general, it is recommended that HDFS and YARN run as separate users.
+  In the majority of installations, HDFS processes execute as 'hdfs'.  YARN
+  is typically using the 'yarn' account.
 
 ** Hadoop Startup
 
-  To start a Hadoop cluster you will need to start both the HDFS and YARN
-  cluster.
+    To start a Hadoop cluster you will need to start both the HDFS and YARN
+    cluster.
 
-  Format a new distributed filesystem:
+    The first time you bring up HDFS, it must be formatted.  Format a new
+    distributed filesystem as <hdfs>:
 
 ----
-$ $HADOOP_PREFIX/bin/hdfs namenode -format <cluster_name>
+[hdfs]$ $HADOOP_PREFIX/bin/hdfs namenode -format <cluster_name>
 ----
 
-  Start the HDFS with the following command, run on the designated NameNode:
+    Start the HDFS NameNode with the following command on the
+    designated node as <hdfs>:
 
 ----
-$ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs start namenode
-----    	
-
-  Run a script to start DataNodes on all slaves:
-
+[hdfs]$ $HADOOP_PREFIX/bin/hdfs --daemon start namenode
 ----
-$ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs start datanode
-----    	
 
-  Start the YARN with the following command, run on the designated
-  ResourceManager:
+    Start a HDFS DataNode with the following command on each
+    designated node as <hdfs>:
 
 ----
-$ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR start resourcemanager
-----    	
-
-  Run a script to start NodeManagers on all slaves:
-
+[hdfs]$ $HADOOP_PREFIX/bin/hdfs --daemon start datanode
 ----
-$ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR start nodemanager
-----    	
 
-  Start a standalone WebAppProxy server.  If multiple servers
-  are used with load balancing it should be run on each of them:
+    If <<<etc/hadoop/slaves>>> and ssh trusted access is configured
+    (see {{{./SingleCluster.html}Single Node Setup}}), all of the
+    HDFS processes can be started with a utility script.  As <hdfs>:
 
 ----
-$ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh start proxyserver --config $HADOOP_CONF_DIR
+[hdfs]$ $HADOOP_PREFIX/sbin/start-dfs.sh
 ----
 
-  Start the MapReduce JobHistory Server with the following command, run on the
-  designated server:
+    Start the YARN with the following command, run on the designated
+    ResourceManager as <yarn>:
 
 ----
-$ $HADOOP_PREFIX/sbin/mr-jobhistory-daemon.sh start historyserver --config $HADOOP_CONF_DIR
-----    	
-
-** Hadoop Shutdown
-
-  Stop the NameNode with the following command, run on the designated
-  NameNode:
-
+[yarn]$ $HADOOP_PREFIX/bin/yarn --daemon start resourcemanager
 ----
-$ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs stop namenode
-----    	
 
-  Run a script to stop DataNodes on all slaves:
+    Run a script to start a NodeManager on each designated host as <yarn>:
 
 ----
-$ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs stop datanode
-----    	
-
-  Stop the ResourceManager with the following command, run on the designated
-  ResourceManager:
-
+[yarn]$ $HADOOP_PREFIX/bin/yarn --daemon start nodemanager
 ----
-$ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR stop resourcemanager
-----    	
 
-  Run a script to stop NodeManagers on all slaves:
+    Start a standalone WebAppProxy server. Run on the WebAppProxy
+    server as <yarn>.  If multiple servers are used with load balancing
+    it should be run on each of them:
 
 ----
-$ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR stop nodemanager
-----    	
+[yarn]$ $HADOOP_PREFIX/bin/yarn --daemon start proxyserver
+----
 
-  Stop the WebAppProxy server. If multiple servers are used with load
-  balancing it should be run on each of them:
+    If <<<etc/hadoop/slaves>>> and ssh trusted access is configured
+    (see {{{./SingleCluster.html}Single Node Setup}}), all of the
+    YARN processes can be started with a utility script.  As <yarn>:
 
 ----
-$ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh stop proxyserver --config $HADOOP_CONF_DIR
+[yarn]$ $HADOOP_PREFIX/sbin/start-yarn.sh
 ----
 
-
-  Stop the MapReduce JobHistory Server with the following command, run on the
-  designated server:
+    Start the MapReduce JobHistory Server with the following command, run
+    on the designated server as <mapred>:
 
 ----
-$ $HADOOP_PREFIX/sbin/mr-jobhistory-daemon.sh stop historyserver --config $HADOOP_CONF_DIR
-----    	
-
-
-* {Operating the Hadoop Cluster}
-
-  Once all the necessary configuration is complete, distribute the files to the
-  <<<HADOOP_CONF_DIR>>> directory on all the machines.
-
-  This section also describes the various Unix users who should be starting the
-  various components and uses the same Unix accounts and groups used previously:
-
-** Hadoop Startup
+[mapred]$ $HADOOP_PREFIX/bin/mapred --daemon start historyserver
+----
 
-    To start a Hadoop cluster you will need to start both the HDFS and YARN
-    cluster.
+** Hadoop Shutdown
 
-    Format a new distributed filesystem as <hdfs>:
+  Stop the NameNode with the following command, run on the designated NameNode
+  as <hdfs>:
 
 ----
-[hdfs]$ $HADOOP_PREFIX/bin/hdfs namenode -format <cluster_name>
+[hdfs]$ $HADOOP_PREFIX/bin/hdfs --daemon stop namenode
 ----
 
-    Start the HDFS with the following command, run on the designated NameNode
-    as <hdfs>:
+  Run a script to stop a DataNode as <hdfs>:
 
 ----
-[hdfs]$ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs start namenode
-----    	
-
-    Run a script to start DataNodes on all slaves as <root> with a special
-    environment variable <<<HADOOP_SECURE_DN_USER>>> set to <hdfs>:
-
+[hdfs]$ $HADOOP_PREFIX/bin/hdfs --daemon stop datanode
 ----
-[root]$ HADOOP_SECURE_DN_USER=hdfs $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs start datanode
-----    	
 
-    Start the YARN with the following command, run on the designated
-    ResourceManager as <yarn>:
+    If <<<etc/hadoop/slaves>>> and ssh trusted access is configured
+    (see {{{./SingleCluster.html}Single Node Setup}}), all of the
+    HDFS processes may be stopped with a utility script.  As <hdfs>:
 
 ----
-[yarn]$ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR start resourcemanager
-----    	
-
-    Run a script to start NodeManagers on all slaves as <yarn>:
-
+[hdfs]$ $HADOOP_PREFIX/sbin/stop-dfs.sh
 ----
-[yarn]$ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR start nodemanager
-----    	
 
-    Start a standalone WebAppProxy server. Run on the WebAppProxy
-    server as <yarn>.  If multiple servers are used with load balancing
-    it should be run on each of them:
+  Stop the ResourceManager with the following command, run on the designated
+  ResourceManager as <yarn>:
 
 ----
-[yarn]$ $HADOOP_YARN_HOME/bin/yarn start proxyserver --config $HADOOP_CONF_DIR
-----    	
-
-    Start the MapReduce JobHistory Server with the following command, run on the
-    designated server as <mapred>:
-
+[yarn]$ $HADOOP_PREFIX/bin/yarn --daemon stop resourcemanager
 ----
-[mapred]$ $HADOOP_PREFIX/sbin/mr-jobhistory-daemon.sh start historyserver --config $HADOOP_CONF_DIR
-----    	
 
-** Hadoop Shutdown
-
-  Stop the NameNode with the following command, run on the designated NameNode
-  as <hdfs>:
+  Run a script to stop a NodeManager on a slave as <yarn>:
 
 ----
-[hdfs]$ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs stop namenode
-----    	
-
-  Run a script to stop DataNodes on all slaves as <root>:
-
+[yarn]$ $HADOOP_PREFIX/bin/yarn --daemon stop nodemanager
 ----
-[root]$ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs stop datanode
-----    	
 
-  Stop the ResourceManager with the following command, run on the designated
-  ResourceManager as <yarn>:
+    If <<<etc/hadoop/slaves>>> and ssh trusted access is configured
+    (see {{{./SingleCluster.html}Single Node Setup}}), all of the
+    YARN processes can be stopped with a utility script.  As <yarn>:
 
 ----
-[yarn]$ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR stop resourcemanager
-----    	
-
-  Run a script to stop NodeManagers on all slaves as <yarn>:
-
+[yarn]$ $HADOOP_PREFIX/sbin/stop-yarn.sh
 ----
-[yarn]$ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR stop nodemanager
-----    	
 
   Stop the WebAppProxy server. Run on the WebAppProxy  server as
   <yarn>.  If multiple servers are used with load balancing it
   should be run on each of them:
 
 ----
-[yarn]$ $HADOOP_YARN_HOME/bin/yarn stop proxyserver --config $HADOOP_CONF_DIR
+[yarn]$ $HADOOP_PREFIX/bin/yarn stop proxyserver
 ----
 
   Stop the MapReduce JobHistory Server with the following command, run on the
   designated server as <mapred>:
 
 ----
-[mapred]$ $HADOOP_PREFIX/sbin/mr-jobhistory-daemon.sh stop historyserver --config $HADOOP_CONF_DIR
-----    	
+[mapred]$ $HADOOP_PREFIX/bin/mapred --daemon stop historyserver
+----
 
 * {Web Interfaces}
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e38cd055/hadoop-common-project/hadoop-common/src/site/apt/CommandsManual.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/apt/CommandsManual.apt.vm b/hadoop-common-project/hadoop-common/src/site/apt/CommandsManual.apt.vm
index 6d2fd5e..67c8bc3 100644
--- a/hadoop-common-project/hadoop-common/src/site/apt/CommandsManual.apt.vm
+++ b/hadoop-common-project/hadoop-common/src/site/apt/CommandsManual.apt.vm
@@ -21,102 +21,161 @@
 
 %{toc}
 
-Overview
+Hadoop Commands Guide
 
-   All hadoop commands are invoked by the <<<bin/hadoop>>> script. Running the
-   hadoop script without any arguments prints the description for all
-   commands.
+* Overview
 
-   Usage: <<<hadoop [--config confdir] [--loglevel loglevel] [COMMAND]
-             [GENERIC_OPTIONS] [COMMAND_OPTIONS]>>>
+   All of the Hadoop commands and subprojects follow the same basic structure:
 
-   Hadoop has an option parsing framework that employs parsing generic
-   options as well as running classes.
+   Usage: <<<shellcommand [SHELL_OPTIONS] [COMMAND] [GENERIC_OPTIONS] [COMMAND_OPTIONS]>>>
 
+*--------+---------+
+|| FIELD || Description 
 *-----------------------+---------------+
-|| COMMAND_OPTION       || Description
+| shellcommand | The command of the project being invoked.  For example,
+               | Hadoop common uses <<<hadoop>>>, HDFS uses <<<hdfs>>>, 
+               | and YARN uses <<<yarn>>>.
+*---------------+-------------------+
+| SHELL_OPTIONS | Options that the shell processes prior to executing Java.
 *-----------------------+---------------+
-| <<<--config confdir>>>| Overwrites the default Configuration directory.  Default is <<<${HADOOP_HOME}/conf>>>.
+| COMMAND | Action to perform.
 *-----------------------+---------------+
-| <<<--loglevel loglevel>>>| Overwrites the log level. Valid log levels are
-|                       | FATAL, ERROR, WARN, INFO, DEBUG, and TRACE.
-|                       | Default is INFO.
+| GENERIC_OPTIONS       | The common set of options supported by 
+                        | multiple commands.
 *-----------------------+---------------+
-| GENERIC_OPTIONS       | The common set of options supported by multiple commands.
-| COMMAND_OPTIONS       | Various commands with their options are described in the following sections. The commands have been grouped into User Commands and Administration Commands.
+| COMMAND_OPTIONS       | Various commands with their options are 
+                        | described in this documention for the 
+                        | Hadoop common sub-project.  HDFS and YARN are
+                        | covered in other documents.
 *-----------------------+---------------+
 
-Generic Options
+** {Shell Options}
 
-   The following options are supported by {{dfsadmin}}, {{fs}}, {{fsck}},
-   {{job}} and {{fetchdt}}. Applications should implement 
-   {{{../../api/org/apache/hadoop/util/Tool.html}Tool}} to support
-   GenericOptions.
+   All of the shell commands will accept a common set of options.  For some commands,
+   these options are ignored. For example, passing <<<---hostnames>>> on a
+   command that only executes on a single host will be ignored.
+
+*-----------------------+---------------+
+|| SHELL_OPTION       || Description
+*-----------------------+---------------+
+| <<<--buildpaths>>>    | Enables developer versions of jars.
+*-----------------------+---------------+
+| <<<--config confdir>>> | Overwrites the default Configuration 
+                         | directory.  Default is <<<${HADOOP_PREFIX}/conf>>>.
+*-----------------------+----------------+
+| <<<--daemon mode>>>   | If the command supports daemonization (e.g.,
+                        | <<<hdfs namenode>>>), execute in the appropriate
+                        | mode. Supported modes are <<<start>>> to start the
+                        | process in daemon mode, <<<stop>>> to stop the
+                        | process, and <<<status>>> to determine the active
+                        | status of the process.  <<<status>>> will return
+                        | an {{{http://refspecs.linuxbase.org/LSB_3.0.0/LSB-generic/LSB-generic/iniscrptact.html}LSB-compliant}} result code. 
+                        | If no option is provided, commands that support
+                        | daemonization will run in the foreground.   
+*-----------------------+---------------+
+| <<<--debug>>>         | Enables shell level configuration debugging information
+*-----------------------+---------------+
+| <<<--help>>>          | Shell script usage information.
+*-----------------------+---------------+
+| <<<--hostnames>>> | A space delimited list of hostnames where to execute 
+                    | a multi-host subcommand. By default, the content of
+                    | the <<<slaves>>> file is used.  
+*-----------------------+----------------+
+| <<<--hosts>>> | A file that contains a list of hostnames where to execute
+                | a multi-host subcommand. By default, the content of the
+                | <<<slaves>>> file is used.  
+*-----------------------+----------------+
+| <<<--loglevel loglevel>>> | Overrides the log level. Valid log levels are
+|                           | FATAL, ERROR, WARN, INFO, DEBUG, and TRACE.
+|                           | Default is INFO.
+*-----------------------+---------------+
+
+** {Generic Options}
+
+   Many subcommands honor a common set of configuration options to alter their behavior:
 
 *------------------------------------------------+-----------------------------+
 ||            GENERIC_OPTION                     ||            Description
 *------------------------------------------------+-----------------------------+
+|<<<-archives \<comma separated list of archives\> >>> | Specify comma separated
+                                                 | archives to be unarchived on
+                                                 | the compute machines. Applies
+                                                 | only to job.
+*------------------------------------------------+-----------------------------+
 |<<<-conf \<configuration file\> >>>             | Specify an application
                                                  | configuration file.
 *------------------------------------------------+-----------------------------+
 |<<<-D \<property\>=\<value\> >>>                | Use value for given property.
 *------------------------------------------------+-----------------------------+
-|<<<-jt \<local\> or \<resourcemanager:port\>>>> | Specify a ResourceManager.
-                                                 | Applies only to job.
-*------------------------------------------------+-----------------------------+
 |<<<-files \<comma separated list of files\> >>> | Specify comma separated files
                                                  | to be copied to the map
                                                  | reduce cluster.  Applies only
                                                  | to job.
 *------------------------------------------------+-----------------------------+
+|<<<-jt \<local\> or \<resourcemanager:port\>>>> | Specify a ResourceManager.
+                                                 | Applies only to job.
+*------------------------------------------------+-----------------------------+
 |<<<-libjars \<comma seperated list of jars\> >>>| Specify comma separated jar
                                                  | files to include in the
                                                  | classpath. Applies only to
                                                  | job.
 *------------------------------------------------+-----------------------------+
-|<<<-archives \<comma separated list of archives\> >>> | Specify comma separated
-                                                 | archives to be unarchived on
-                                                 | the compute machines. Applies
-                                                 | only to job.
-*------------------------------------------------+-----------------------------+
 
-User Commands
+Hadoop Common Commands
 
-   Commands useful for users of a hadoop cluster.
+  All of these commands are executed from the <<<hadoop>>> shell command.  They
+  have been broken up into {{User Commands}} and 
+  {{Admininistration Commands}}.
+
+* User Commands
 
-* <<<archive>>>
+   Commands useful for users of a hadoop cluster.
 
+** <<<archive>>>
+    
    Creates a hadoop archive. More information can be found at
-   {{{../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/HadoopArchives.html}
+  {{{../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/HadoopArchives.html}
    Hadoop Archives Guide}}.
 
-* <<<credential>>>
+** <<<checknative>>>
 
-   Command to manage credentials, passwords and secrets within credential providers.
+    Usage: <<<hadoop checknative [-a] [-h] >>>
 
-   The CredentialProvider API in Hadoop allows for the separation of applications
-   and how they store their required passwords/secrets. In order to indicate
-   a particular provider type and location, the user must provide the
-   <hadoop.security.credential.provider.path> configuration element in core-site.xml
-   or use the command line option <<<-provider>>> on each of the following commands.
-   This provider path is a comma-separated list of URLs that indicates the type and
-   location of a list of providers that should be consulted.
-   For example, the following path:
+*-----------------+-----------------------------------------------------------+
+|| COMMAND_OPTION || Description
+*-----------------+-----------------------------------------------------------+
+| -a              | Check all libraries are available.
+*-----------------+-----------------------------------------------------------+
+| -h              | print help
+*-----------------+-----------------------------------------------------------+
 
-   <<<user:///,jceks://file/tmp/test.jceks,jceks://hdfs@nn1.example.com/my/path/test.jceks>>>
+    This command checks the availability of the Hadoop native code.  See
+    {{{NativeLibraries.html}}} for more information.  By default, this command 
+    only checks the availability of libhadoop.
 
-   indicates that the current user's credentials file should be consulted through
-   the User Provider, that the local file located at <<</tmp/test.jceks>>> is a Java Keystore
-   Provider and that the file located within HDFS at <<<nn1.example.com/my/path/test.jceks>>>
-   is also a store for a Java Keystore Provider.
+** <<<classpath>>>
 
-   When utilizing the credential command it will often be for provisioning a password
-   or secret to a particular credential store provider. In order to explicitly
-   indicate which provider store to use the <<<-provider>>> option should be used. Otherwise,
-   given a path of multiple providers, the first non-transient provider will be used.
-   This may or may not be the one that you intended.
+   Usage: <<<hadoop classpath [--glob|--jar <path>|-h|--help]>>>
 
-   Example: <<<-provider jceks://file/tmp/test.jceks>>>
+*-----------------+-----------------------------------------------------------+
+|| COMMAND_OPTION || Description
+*-----------------+-----------------------------------------------------------+
+| --glob          | expand wildcards
+*-----------------+-----------------------------------------------------------+
+| --jar <path>    | write classpath as manifest in jar named <path>
+*-----------------+-----------------------------------------------------------+
+| -h, --help      | print help
+*-----------------+-----------------------------------------------------------+
+
+   Prints the class path needed to get the Hadoop jar and the required
+   libraries.  If called without arguments, then prints the classpath set up by
+   the command scripts, which is likely to contain wildcards in the classpath
+   entries.  Additional options print the classpath after wildcard expansion or
+   write the classpath into the manifest of a jar file.  The latter is useful in
+   environments where wildcards cannot be used and the expanded classpath exceeds
+   the maximum supported command line length.
+
+** <<<credential>>>
 
    Usage: <<<hadoop credential <subcommand> [options]>>>
 
@@ -143,109 +202,96 @@ User Commands
                     | indicated.
 *-------------------+-------------------------------------------------------+
 
-* <<<distcp>>>
+   Command to manage credentials, passwords and secrets within credential providers.
 
-   Copy file or directories recursively. More information can be found at
-   {{{../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/DistCp.html}
-   Hadoop DistCp Guide}}.
+   The CredentialProvider API in Hadoop allows for the separation of applications
+   and how they store their required passwords/secrets. In order to indicate
+   a particular provider type and location, the user must provide the
+   <hadoop.security.credential.provider.path> configuration element in core-site.xml
+   or use the command line option <<<-provider>>> on each of the following commands.
+   This provider path is a comma-separated list of URLs that indicates the type and
+   location of a list of providers that should be consulted. For example, the following path:
+   <<<user:///,jceks://file/tmp/test.jceks,jceks://hdfs@nn1.example.com/my/path/test.jceks>>>
 
-* <<<fs>>>
+   indicates that the current user's credentials file should be consulted through
+   the User Provider, that the local file located at <<</tmp/test.jceks>>> is a Java Keystore
+   Provider and that the file located within HDFS at <<<nn1.example.com/my/path/test.jceks>>>
+   is also a store for a Java Keystore Provider.
 
-   Deprecated, use {{{../hadoop-hdfs/HDFSCommands.html#dfs}<<<hdfs dfs>>>}}
-   instead.
+   When utilizing the credential command it will often be for provisioning a password
+   or secret to a particular credential store provider. In order to explicitly
+   indicate which provider store to use the <<<-provider>>> option should be used. Otherwise,
+   given a path of multiple providers, the first non-transient provider will be used.
+   This may or may not be the one that you intended.
 
-* <<<fsck>>>
+   Example: <<<-provider jceks://file/tmp/test.jceks>>>
 
-   Deprecated, use {{{../hadoop-hdfs/HDFSCommands.html#fsck}<<<hdfs fsck>>>}}
-   instead.
+** <<<distch>>>
 
-* <<<fetchdt>>>
+  Usage: <<<hadoop distch [-f urilist_url] [-i] [-log logdir] path:owner:group:permissions>>>
+  
+*-------------------+-------------------------------------------------------+
+||COMMAND_OPTION    ||                   Description
+*-------------------+-------------------------------------------------------+
+| -f | List of objects to change
+*----+------------+
+| -i | Ignore failures
+*----+------------+
+| -log | Directory to log output
+*-----+---------+
 
-   Deprecated, use {{{../hadoop-hdfs/HDFSCommands.html#fetchdt}
-   <<<hdfs fetchdt>>>}} instead.
+  Change the ownership and permissions on many files at once.
 
-* <<<jar>>>
+** <<<distcp>>>
 
-   Runs a jar file. Users can bundle their Map Reduce code in a jar file and
-   execute it using this command.
+   Copy file or directories recursively. More information can be found at
+   {{{../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/DistCp.html}
+   Hadoop DistCp Guide}}.
 
-   Usage: <<<hadoop jar <jar> [mainClass] args...>>>
+** <<<fs>>>
 
-   The streaming jobs are run via this command. Examples can be referred from
-   Streaming examples
+   This command is documented in the {{{./FileSystemShell.html}File System Shell Guide}}.  It is a synonym for <<<hdfs dfs>>> when HDFS is in use.
 
-   Word count example is also run using jar command. It can be referred from
-   Wordcount example
+** <<<jar>>>
 
-   Use {{{../../hadoop-yarn/hadoop-yarn-site/YarnCommands.html#jar}<<<yarn jar>>>}}
-   to launch YARN applications instead.
+  Usage: <<<hadoop jar <jar> [mainClass] args...>>>
 
-* <<<job>>>
+  Runs a jar file. 
+  
+  Use {{{../../hadoop-yarn/hadoop-yarn-site/YarnCommands.html#jar}<<<yarn jar>>>}}
+  to launch YARN applications instead.
 
-   Deprecated. Use
-   {{{../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapredCommands.html#job}
-   <<<mapred job>>>}} instead.
+** <<<jnipath>>>
 
-* <<<pipes>>>
+    Usage: <<<hadoop jnipath>>>
 
-   Deprecated. Use
-   {{{../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapredCommands.html#pipes}
-   <<<mapred pipes>>>}} instead.
+    Print the computed java.library.path.
 
-* <<<queue>>>
+** <<<key>>>
 
-   Deprecated. Use
-   {{{../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapredCommands.html#queue}
-   <<<mapred queue>>>}} instead.
+    Manage keys via the KeyProvider.
 
-* <<<version>>>
+** <<<trace>>>
 
-   Prints the version.
+    View and modify Hadoop tracing settings.   See the {{{./Tracing.html}Tracing Guide}}.
+
+** <<<version>>>
 
    Usage: <<<hadoop version>>>
 
-* <<<CLASSNAME>>>
+   Prints the version.
 
-   hadoop script can be used to invoke any class.
+** <<<CLASSNAME>>>
 
    Usage: <<<hadoop CLASSNAME>>>
 
-   Runs the class named <<<CLASSNAME>>>.
-
-* <<<classpath>>>
-
-   Prints the class path needed to get the Hadoop jar and the required
-   libraries.  If called without arguments, then prints the classpath set up by
-   the command scripts, which is likely to contain wildcards in the classpath
-   entries.  Additional options print the classpath after wildcard expansion or
-   write the classpath into the manifest of a jar file.  The latter is useful in
-   environments where wildcards cannot be used and the expanded classpath exceeds
-   the maximum supported command line length.
+   Runs the class named <<<CLASSNAME>>>.  The class must be part of a package.
 
-   Usage: <<<hadoop classpath [--glob|--jar <path>|-h|--help]>>>
-
-*-----------------+-----------------------------------------------------------+
-|| COMMAND_OPTION || Description
-*-----------------+-----------------------------------------------------------+
-| --glob          | expand wildcards
-*-----------------+-----------------------------------------------------------+
-| --jar <path>    | write classpath as manifest in jar named <path>
-*-----------------+-----------------------------------------------------------+
-| -h, --help      | print help
-*-----------------+-----------------------------------------------------------+
-
-Administration Commands
+* {Administration Commands}
 
    Commands useful for administrators of a hadoop cluster.
 
-* <<<balancer>>>
-
-   Deprecated, use {{{../hadoop-hdfs/HDFSCommands.html#balancer}
-   <<<hdfs balancer>>>}} instead.
-
-* <<<daemonlog>>>
-
-   Get/Set the log level for each daemon.
+** <<<daemonlog>>>
 
    Usage: <<<hadoop daemonlog -getlevel <host:port> <name> >>>
    Usage: <<<hadoop daemonlog -setlevel <host:port> <name> <level> >>>
@@ -262,22 +308,20 @@ Administration Commands
                                | connects to http://<host:port>/logLevel?log=<name>
 *------------------------------+-----------------------------------------------------------+
 
-* <<<datanode>>>
+   Get/Set the log level for each daemon.
 
-   Deprecated, use {{{../hadoop-hdfs/HDFSCommands.html#datanode}
-   <<<hdfs datanode>>>}} instead.
+* Files
 
-* <<<dfsadmin>>>
+** <<etc/hadoop/hadoop-env.sh>>
 
-   Deprecated, use {{{../hadoop-hdfs/HDFSCommands.html#dfsadmin}
-   <<<hdfs dfsadmin>>>}} instead.
+    This file stores the global settings used by all Hadoop shell commands.
 
-* <<<namenode>>>
+** <<etc/hadoop/hadoop-user-functions.sh>>
 
-   Deprecated, use {{{../hadoop-hdfs/HDFSCommands.html#namenode}
-   <<<hdfs namenode>>>}} instead.
+    This file allows for advanced users to override some shell functionality.
 
-* <<<secondarynamenode>>>
+** <<~/.hadooprc>>
 
-   Deprecated, use {{{../hadoop-hdfs/HDFSCommands.html#secondarynamenode}
-   <<<hdfs secondarynamenode>>>}} instead.
+    This stores the personal environment for an individual user.  It is
+    processed after the hadoop-env.sh and hadoop-user-functions.sh files
+    and can contain the same settings.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e38cd055/hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm b/hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm
index 1a9618c..757a0ba 100644
--- a/hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm
+++ b/hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm
@@ -45,46 +45,62 @@ bin/hadoop fs <args>
    Differences are described with each of the commands. Error information is
    sent to stderr and the output is sent to stdout.
 
-appendToFile
+   If HDFS is being used, <<<hdfs dfs>>> is a synonym.
 
-      Usage: <<<hdfs dfs -appendToFile <localsrc> ... <dst> >>>
+   See the {{{./CommandsManual.html}Commands Manual}} for generic shell options.
+
+* appendToFile
+
+      Usage: <<<hadoop fs -appendToFile <localsrc> ... <dst> >>>
 
       Append single src, or multiple srcs from local file system to the
       destination file system. Also reads input from stdin and appends to
       destination file system.
 
-        * <<<hdfs dfs -appendToFile localfile /user/hadoop/hadoopfile>>>
+        * <<<hadoop fs -appendToFile localfile /user/hadoop/hadoopfile>>>
 
-        * <<<hdfs dfs -appendToFile localfile1 localfile2 /user/hadoop/hadoopfile>>>
+        * <<<hadoop fs -appendToFile localfile1 localfile2 /user/hadoop/hadoopfile>>>
 
-        * <<<hdfs dfs -appendToFile localfile hdfs://nn.example.com/hadoop/hadoopfile>>>
+        * <<<hadoop fs -appendToFile localfile hdfs://nn.example.com/hadoop/hadoopfile>>>
 
-        * <<<hdfs dfs -appendToFile - hdfs://nn.example.com/hadoop/hadoopfile>>>
+        * <<<hadoop fs -appendToFile - hdfs://nn.example.com/hadoop/hadoopfile>>>
           Reads the input from stdin.
 
       Exit Code:
 
       Returns 0 on success and 1 on error.
 
-cat
+* cat
 
-   Usage: <<<hdfs dfs -cat URI [URI ...]>>>
+   Usage: <<<hadoop fs -cat URI [URI ...]>>>
 
    Copies source paths to stdout.
 
    Example:
 
-     * <<<hdfs dfs -cat hdfs://nn1.example.com/file1 hdfs://nn2.example.com/file2>>>
+     * <<<hadoop fs -cat hdfs://nn1.example.com/file1 hdfs://nn2.example.com/file2>>>
 
-     * <<<hdfs dfs -cat file:///file3 /user/hadoop/file4>>>
+     * <<<hadoop fs -cat file:///file3 /user/hadoop/file4>>>
 
    Exit Code:
 
    Returns 0 on success and -1 on error.
 
-chgrp
+* checksum
+
+  Usage: <<<hadoop fs -checksum URI>>>
+
+  Returns the checksum information of a file.
+
+  Example:
+
+    * <<<hadoop fs -checksum hdfs://nn1.example.com/file1>>>
 
-   Usage: <<<hdfs dfs -chgrp [-R] GROUP URI [URI ...]>>>
+    * <<<hadoop fs -checksum file:///etc/hosts>>>
+
+* chgrp
+
+   Usage: <<<hadoop fs -chgrp [-R] GROUP URI [URI ...]>>>
 
    Change group association of files. The user must be the owner of files, or
    else a super-user. Additional information is in the
@@ -94,9 +110,9 @@ chgrp
 
      * The -R option will make the change recursively through the directory structure.
 
-chmod
+* chmod
 
-   Usage: <<<hdfs dfs -chmod [-R] <MODE[,MODE]... | OCTALMODE> URI [URI ...]>>>
+   Usage: <<<hadoop fs -chmod [-R] <MODE[,MODE]... | OCTALMODE> URI [URI ...]>>>
 
    Change the permissions of files. With -R, make the change recursively
    through the directory structure. The user must be the owner of the file, or
@@ -107,9 +123,9 @@ chmod
 
      * The -R option will make the change recursively through the directory structure.
 
-chown
+* chown
 
-   Usage: <<<hdfs dfs -chown [-R] [OWNER][:[GROUP]] URI [URI ]>>>
+   Usage: <<<hadoop fs -chown [-R] [OWNER][:[GROUP]] URI [URI ]>>>
 
    Change the owner of files. The user must be a super-user. Additional information
    is in the {{{../hadoop-hdfs/HdfsPermissionsGuide.html}Permissions Guide}}.
@@ -118,9 +134,9 @@ chown
 
      * The -R option will make the change recursively through the directory structure.
 
-copyFromLocal
+* copyFromLocal
 
-   Usage: <<<hdfs dfs -copyFromLocal <localsrc> URI>>>
+   Usage: <<<hadoop fs -copyFromLocal <localsrc> URI>>>
 
    Similar to put command, except that the source is restricted to a local
    file reference.
@@ -129,16 +145,16 @@ copyFromLocal
 
      * The -f option will overwrite the destination if it already exists.
 
-copyToLocal
+* copyToLocal
 
-   Usage: <<<hdfs dfs -copyToLocal [-ignorecrc] [-crc] URI <localdst> >>>
+   Usage: <<<hadoop fs -copyToLocal [-ignorecrc] [-crc] URI <localdst> >>>
 
    Similar to get command, except that the destination is restricted to a
    local file reference.
 
-count
+* count
 
-   Usage: <<<hdfs dfs -count [-q] [-h] <paths> >>>
+   Usage: <<<hadoop fs -count [-q] [-h] <paths> >>>
 
    Count the number of directories, files and bytes under the paths that match
    the specified file pattern.  The output columns with -count are: DIR_COUNT,
@@ -151,19 +167,19 @@ count
 
    Example:
 
-     * <<<hdfs dfs -count hdfs://nn1.example.com/file1 hdfs://nn2.example.com/file2>>>
+     * <<<hadoop fs -count hdfs://nn1.example.com/file1 hdfs://nn2.example.com/file2>>>
 
-     * <<<hdfs dfs -count -q hdfs://nn1.example.com/file1>>>
+     * <<<hadoop fs -count -q hdfs://nn1.example.com/file1>>>
 
-     * <<<hdfs dfs -count -q -h hdfs://nn1.example.com/file1>>>
+     * <<<hadoop fs -count -q -h hdfs://nn1.example.com/file1>>>
 
    Exit Code:
 
    Returns 0 on success and -1 on error.
 
-cp
+* cp
 
-   Usage: <<<hdfs dfs -cp [-f] [-p | -p[topax]] URI [URI ...] <dest> >>>
+   Usage: <<<hadoop fs -cp [-f] [-p | -p[topax]] URI [URI ...] <dest> >>>
 
    Copy files from source to destination. This command allows multiple sources
    as well in which case the destination must be a directory.
@@ -177,7 +193,7 @@ cp
     Options:
 
       * The -f option will overwrite the destination if it already exists.
-      
+
       * The -p option will preserve file attributes [topx] (timestamps,
         ownership, permission, ACL, XAttr). If -p is specified with no <arg>,
         then preserves timestamps, ownership, permission. If -pa is specified,
@@ -187,17 +203,41 @@ cp
 
    Example:
 
-     * <<<hdfs dfs -cp /user/hadoop/file1 /user/hadoop/file2>>>
+     * <<<hadoop fs -cp /user/hadoop/file1 /user/hadoop/file2>>>
 
-     * <<<hdfs dfs -cp /user/hadoop/file1 /user/hadoop/file2 /user/hadoop/dir>>>
+     * <<<hadoop fs -cp /user/hadoop/file1 /user/hadoop/file2 /user/hadoop/dir>>>
 
    Exit Code:
 
    Returns 0 on success and -1 on error.
 
-du
+* createSnapshot
+
+  See {{{../hadoop-hdfs/HdfsSnapshots.html}HDFS Snapshots Guide}}.
+
+
+* deleteSnapshot
+
+  See {{{../hadoop-hdfs/HdfsSnapshots.html}HDFS Snapshots Guide}}.
+
+* df
+
+   Usage: <<<hadoop fs -df [-h] URI [URI ...]>>>
+
+   Displays free space.
+
+   Options:
+
+     * The -h option will format file sizes in a "human-readable" fashion (e.g
+       64.0m instead of 67108864)
+
+   Example:
 
-   Usage: <<<hdfs dfs -du [-s] [-h] URI [URI ...]>>>
+     * <<<hadoop dfs -df /user/hadoop/dir1>>>
+
+* du
+
+   Usage: <<<hadoop fs -du [-s] [-h] URI [URI ...]>>>
 
    Displays sizes of files and directories contained in the given directory or
    the length of a file in case its just a file.
@@ -212,29 +252,29 @@ du
 
    Example:
 
-    * hdfs dfs -du /user/hadoop/dir1 /user/hadoop/file1 hdfs://nn.example.com/user/hadoop/dir1
+    * <<<hadoop fs -du /user/hadoop/dir1 /user/hadoop/file1 hdfs://nn.example.com/user/hadoop/dir1>>>
 
    Exit Code:
    Returns 0 on success and -1 on error.
 
-dus
+* dus
 
-   Usage: <<<hdfs dfs -dus <args> >>>
+   Usage: <<<hadoop fs -dus <args> >>>
 
    Displays a summary of file lengths.
 
-   <<Note:>> This command is deprecated. Instead use <<<hdfs dfs -du -s>>>.
+   <<Note:>> This command is deprecated. Instead use <<<hadoop fs -du -s>>>.
 
-expunge
+* expunge
 
-   Usage: <<<hdfs dfs -expunge>>>
+   Usage: <<<hadoop fs -expunge>>>
 
    Empty the Trash. Refer to the {{{../hadoop-hdfs/HdfsDesign.html}
    HDFS Architecture Guide}} for more information on the Trash feature.
 
-find
+* find
 
-   Usage: <<<hdfs dfs -find <path> ... <expression> ... >>>
+   Usage: <<<hadoop fs -find <path> ... <expression> ... >>>
 
    Finds all files that match the specified expression and applies selected
    actions to them. If no <path> is specified then defaults to the current
@@ -269,15 +309,15 @@ find
 
    Example:
 
-   <<<hdfs dfs -find / -name test -print>>>
+   <<<hadoop fs -find / -name test -print>>>
 
    Exit Code:
 
      Returns 0 on success and -1 on error.
 
-get
+* get
 
-   Usage: <<<hdfs dfs -get [-ignorecrc] [-crc] <src> <localdst> >>>
+   Usage: <<<hadoop fs -get [-ignorecrc] [-crc] <src> <localdst> >>>
 
    Copy files to the local file system. Files that fail the CRC check may be
    copied with the -ignorecrc option. Files and CRCs may be copied using the
@@ -285,17 +325,17 @@ get
 
    Example:
 
-     * <<<hdfs dfs -get /user/hadoop/file localfile>>>
+     * <<<hadoop fs -get /user/hadoop/file localfile>>>
 
-     * <<<hdfs dfs -get hdfs://nn.example.com/user/hadoop/file localfile>>>
+     * <<<hadoop fs -get hdfs://nn.example.com/user/hadoop/file localfile>>>
 
    Exit Code:
 
    Returns 0 on success and -1 on error.
 
-getfacl
+* getfacl
 
-   Usage: <<<hdfs dfs -getfacl [-R] <path> >>>
+   Usage: <<<hadoop fs -getfacl [-R] <path> >>>
 
    Displays the Access Control Lists (ACLs) of files and directories. If a
    directory has a default ACL, then getfacl also displays the default ACL.
@@ -308,17 +348,17 @@ getfacl
 
    Examples:
 
-     * <<<hdfs dfs -getfacl /file>>>
+     * <<<hadoop fs -getfacl /file>>>
 
-     * <<<hdfs dfs -getfacl -R /dir>>>
+     * <<<hadoop fs -getfacl -R /dir>>>
 
    Exit Code:
 
    Returns 0 on success and non-zero on error.
 
-getfattr
+* getfattr
 
-   Usage: <<<hdfs dfs -getfattr [-R] {-n name | -d} [-e en] <path> >>>
+   Usage: <<<hadoop fs -getfattr [-R] {-n name | -d} [-e en] <path> >>>
 
    Displays the extended attribute names and values (if any) for a file or
    directory.
@@ -337,26 +377,32 @@ getfattr
 
    Examples:
 
-     * <<<hdfs dfs -getfattr -d /file>>>
+     * <<<hadoop fs -getfattr -d /file>>>
 
-     * <<<hdfs dfs -getfattr -R -n user.myAttr /dir>>>
+     * <<<hadoop fs -getfattr -R -n user.myAttr /dir>>>
 
    Exit Code:
 
    Returns 0 on success and non-zero on error.
 
-getmerge
+* getmerge
 
-   Usage: <<<hdfs dfs -getmerge <src> <localdst> [addnl]>>>
+   Usage: <<<hadoop fs -getmerge <src> <localdst> [addnl]>>>
 
    Takes a source directory and a destination file as input and concatenates
    files in src into the destination local file. Optionally addnl can be set to
    enable adding a newline character at the
    end of each file.
 
-ls
+* help
+
+   Usage: <<<hadoop fs -help>>>
 
-   Usage: <<<hdfs dfs -ls [-R] <args> >>>
+   Return usage output.
+
+* ls
+
+   Usage: <<<hadoop fs -ls [-R] <args> >>>
 
    Options:
 
@@ -377,23 +423,23 @@ permissions userid groupid modification_date modification_time dirname
 
    Example:
 
-     * <<<hdfs dfs -ls /user/hadoop/file1>>>
+     * <<<hadoop fs -ls /user/hadoop/file1>>>
 
    Exit Code:
 
    Returns 0 on success and -1 on error.
 
-lsr
+* lsr
 
-   Usage: <<<hdfs dfs -lsr <args> >>>
+   Usage: <<<hadoop fs -lsr <args> >>>
 
    Recursive version of ls.
 
-   <<Note:>> This command is deprecated. Instead use <<<hdfs dfs -ls -R>>>
+   <<Note:>> This command is deprecated. Instead use <<<hadoop fs -ls -R>>>
 
-mkdir
+* mkdir
 
-   Usage: <<<hdfs dfs -mkdir [-p] <paths> >>>
+   Usage: <<<hadoop fs -mkdir [-p] <paths> >>>
 
    Takes path uri's as argument and creates directories.
 
@@ -403,30 +449,30 @@ mkdir
 
    Example:
 
-     * <<<hdfs dfs -mkdir /user/hadoop/dir1 /user/hadoop/dir2>>>
+     * <<<hadoop fs -mkdir /user/hadoop/dir1 /user/hadoop/dir2>>>
 
-     * <<<hdfs dfs -mkdir hdfs://nn1.example.com/user/hadoop/dir hdfs://nn2.example.com/user/hadoop/dir>>>
+     * <<<hadoop fs -mkdir hdfs://nn1.example.com/user/hadoop/dir hdfs://nn2.example.com/user/hadoop/dir>>>
 
    Exit Code:
 
    Returns 0 on success and -1 on error.
 
-moveFromLocal
+* moveFromLocal
 
-   Usage: <<<hdfs dfs -moveFromLocal <localsrc> <dst> >>>
+   Usage: <<<hadoop fs -moveFromLocal <localsrc> <dst> >>>
 
    Similar to put command, except that the source localsrc is deleted after
    it's copied.
 
-moveToLocal
+* moveToLocal
 
-   Usage: <<<hdfs dfs -moveToLocal [-crc] <src> <dst> >>>
+   Usage: <<<hadoop fs -moveToLocal [-crc] <src> <dst> >>>
 
    Displays a "Not implemented yet" message.
 
-mv
+* mv
 
-   Usage: <<<hdfs dfs -mv URI [URI ...] <dest> >>>
+   Usage: <<<hadoop fs -mv URI [URI ...] <dest> >>>
 
    Moves files from source to destination. This command allows multiple sources
    as well in which case the destination needs to be a directory. Moving files
@@ -434,38 +480,42 @@ mv
 
    Example:
 
-     * <<<hdfs dfs -mv /user/hadoop/file1 /user/hadoop/file2>>>
+     * <<<hadoop fs -mv /user/hadoop/file1 /user/hadoop/file2>>>
 
-     * <<<hdfs dfs -mv hdfs://nn.example.com/file1 hdfs://nn.example.com/file2 hdfs://nn.example.com/file3 hdfs://nn.example.com/dir1>>>
+     * <<<hadoop fs -mv hdfs://nn.example.com/file1 hdfs://nn.example.com/file2 hdfs://nn.example.com/file3 hdfs://nn.example.com/dir1>>>
 
    Exit Code:
 
    Returns 0 on success and -1 on error.
 
-put
+* put
 
-   Usage: <<<hdfs dfs -put <localsrc> ... <dst> >>>
+   Usage: <<<hadoop fs -put <localsrc> ... <dst> >>>
 
    Copy single src, or multiple srcs from local file system to the destination
    file system. Also reads input from stdin and writes to destination file
    system.
 
-     * <<<hdfs dfs -put localfile /user/hadoop/hadoopfile>>>
+     * <<<hadoop fs -put localfile /user/hadoop/hadoopfile>>>
 
-     * <<<hdfs dfs -put localfile1 localfile2 /user/hadoop/hadoopdir>>>
+     * <<<hadoop fs -put localfile1 localfile2 /user/hadoop/hadoopdir>>>
 
-     * <<<hdfs dfs -put localfile hdfs://nn.example.com/hadoop/hadoopfile>>>
+     * <<<hadoop fs -put localfile hdfs://nn.example.com/hadoop/hadoopfile>>>
 
-     * <<<hdfs dfs -put - hdfs://nn.example.com/hadoop/hadoopfile>>>
+     * <<<hadoop fs -put - hdfs://nn.example.com/hadoop/hadoopfile>>>
        Reads the input from stdin.
 
    Exit Code:
 
    Returns 0 on success and -1 on error.
 
-rm
+* renameSnapshot
+
+  See {{{../hadoop-hdfs/HdfsSnapshots.html}HDFS Snapshots Guide}}.
 
-   Usage: <<<hdfs dfs -rm [-f] [-r|-R] [-skipTrash] URI [URI ...]>>>
+* rm
+
+   Usage: <<<hadoop fs -rm [-f] [-r|-R] [-skipTrash] URI [URI ...]>>>
 
    Delete files specified as args.
 
@@ -484,23 +534,37 @@ rm
 
    Example:
 
-     * <<<hdfs dfs -rm hdfs://nn.example.com/file /user/hadoop/emptydir>>>
+     * <<<hadoop fs -rm hdfs://nn.example.com/file /user/hadoop/emptydir>>>
 
    Exit Code:
 
    Returns 0 on success and -1 on error.
 
-rmr
+* rmdir
+
+   Usage: <<<hadoop fs -rmdir [--ignore-fail-on-non-empty] URI [URI ...]>>>
+
+   Delete a directory.
+
+   Options:
+
+     * --ignore-fail-on-non-empty: When using wildcards, do not fail if a directory still contains files.
+
+   Example:
+
+     * <<<hadoop fs -rmdir /user/hadoop/emptydir>>>
+
+* rmr
 
-   Usage: <<<hdfs dfs -rmr [-skipTrash] URI [URI ...]>>>
+   Usage: <<<hadoop fs -rmr [-skipTrash] URI [URI ...]>>>
 
    Recursive version of delete.
 
-   <<Note:>> This command is deprecated. Instead use <<<hdfs dfs -rm -r>>>
+   <<Note:>> This command is deprecated. Instead use <<<hadoop fs -rm -r>>>
 
-setfacl
+* setfacl
 
-   Usage: <<<hdfs dfs -setfacl [-R] [{-b|-k} {-m|-x <acl_spec>} <path>]|[--set <acl_spec> <path>] >>>
+   Usage: <<<hadoop fs -setfacl [-R] [{-b|-k} {-m|-x <acl_spec>} <path>]|[--set <acl_spec> <path>] >>>
 
    Sets Access Control Lists (ACLs) of files and directories.
 
@@ -528,27 +592,27 @@ setfacl
 
    Examples:
 
-      * <<<hdfs dfs -setfacl -m user:hadoop:rw- /file>>>
+      * <<<hadoop fs -setfacl -m user:hadoop:rw- /file>>>
 
-      * <<<hdfs dfs -setfacl -x user:hadoop /file>>>
+      * <<<hadoop fs -setfacl -x user:hadoop /file>>>
 
-      * <<<hdfs dfs -setfacl -b /file>>>
+      * <<<hadoop fs -setfacl -b /file>>>
 
-      * <<<hdfs dfs -setfacl -k /dir>>>
+      * <<<hadoop fs -setfacl -k /dir>>>
 
-      * <<<hdfs dfs -setfacl --set user::rw-,user:hadoop:rw-,group::r--,other::r-- /file>>>
+      * <<<hadoop fs -setfacl --set user::rw-,user:hadoop:rw-,group::r--,other::r-- /file>>>
 
-      * <<<hdfs dfs -setfacl -R -m user:hadoop:r-x /dir>>>
+      * <<<hadoop fs -setfacl -R -m user:hadoop:r-x /dir>>>
 
-      * <<<hdfs dfs -setfacl -m default:user:hadoop:r-x /dir>>>
+      * <<<hadoop fs -setfacl -m default:user:hadoop:r-x /dir>>>
 
    Exit Code:
 
    Returns 0 on success and non-zero on error.
 
-setfattr
+* setfattr
 
-   Usage: <<<hdfs dfs -setfattr {-n name [-v value] | -x name} <path> >>>
+   Usage: <<<hadoop fs -setfattr {-n name [-v value] | -x name} <path> >>>
 
    Sets an extended attribute name and value for a file or directory.
 
@@ -566,19 +630,19 @@ setfattr
 
    Examples:
 
-      * <<<hdfs dfs -setfattr -n user.myAttr -v myValue /file>>>
+      * <<<hadoop fs -setfattr -n user.myAttr -v myValue /file>>>
 
-      * <<<hdfs dfs -setfattr -n user.noValue /file>>>
+      * <<<hadoop fs -setfattr -n user.noValue /file>>>
 
-      * <<<hdfs dfs -setfattr -x user.myAttr /file>>>
+      * <<<hadoop fs -setfattr -x user.myAttr /file>>>
 
    Exit Code:
 
    Returns 0 on success and non-zero on error.
 
-setrep
+* setrep
 
-   Usage: <<<hdfs dfs -setrep [-R] [-w] <numReplicas> <path> >>>
+   Usage: <<<hadoop fs -setrep [-R] [-w] <numReplicas> <path> >>>
 
    Changes the replication factor of a file. If <path> is a directory then
    the command recursively changes the replication factor of all files under
@@ -593,28 +657,28 @@ setrep
 
    Example:
 
-     * <<<hdfs dfs -setrep -w 3 /user/hadoop/dir1>>>
+     * <<<hadoop fs -setrep -w 3 /user/hadoop/dir1>>>
 
    Exit Code:
 
    Returns 0 on success and -1 on error.
 
-stat
+* stat
 
-   Usage: <<<hdfs dfs -stat URI [URI ...]>>>
+   Usage: <<<hadoop fs -stat URI [URI ...]>>>
 
    Returns the stat information on the path.
 
    Example:
 
-     * <<<hdfs dfs -stat path>>>
+     * <<<hadoop fs -stat path>>>
 
    Exit Code:
    Returns 0 on success and -1 on error.
 
-tail
+* tail
 
-   Usage: <<<hdfs dfs -tail [-f] URI>>>
+   Usage: <<<hadoop fs -tail [-f] URI>>>
 
    Displays last kilobyte of the file to stdout.
 
@@ -624,43 +688,54 @@ tail
 
    Example:
 
-     * <<<hdfs dfs -tail pathname>>>
+     * <<<hadoop fs -tail pathname>>>
 
    Exit Code:
    Returns 0 on success and -1 on error.
 
-test
+* test
 
-   Usage: <<<hdfs dfs -test -[ezd] URI>>>
+   Usage: <<<hadoop fs -test -[defsz] URI>>>
 
    Options:
 
-     * The -e option will check to see if the file exists, returning 0 if true.
+     * -d: f the path is a directory, return 0.
+
+     * -e: if the path exists, return 0.
 
-     * The -z option will check to see if the file is zero length, returning 0 if true.
+     * -f: if the path is a file, return 0.
 
-     * The -d option will check to see if the path is directory, returning 0 if true.
+     * -s: if the path is not empty, return 0.
+
+     * -z: if the file is zero length, return 0.
 
    Example:
 
-     * <<<hdfs dfs -test -e filename>>>
+     * <<<hadoop fs -test -e filename>>>
 
-text
+* text
 
-   Usage: <<<hdfs dfs -text <src> >>>
+   Usage: <<<hadoop fs -text <src> >>>
 
    Takes a source file and outputs the file in text format. The allowed formats
    are zip and TextRecordInputStream.
 
-touchz
+* touchz
 
-   Usage: <<<hdfs dfs -touchz URI [URI ...]>>>
+   Usage: <<<hadoop fs -touchz URI [URI ...]>>>
 
    Create a file of zero length.
 
    Example:
 
-     * <<<hdfs dfs -touchz pathname>>>
+     * <<<hadoop fs -touchz pathname>>>
 
    Exit Code:
    Returns 0 on success and -1 on error.
+
+
+* usage
+
+   Usage: <<<hadoop fs -usage command>>>
+
+   Return the help for an individual command.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e38cd055/hadoop-common-project/hadoop-common/src/site/apt/SingleCluster.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/apt/SingleCluster.apt.vm b/hadoop-common-project/hadoop-common/src/site/apt/SingleCluster.apt.vm
index ef7532a..eb9c88a 100644
--- a/hadoop-common-project/hadoop-common/src/site/apt/SingleCluster.apt.vm
+++ b/hadoop-common-project/hadoop-common/src/site/apt/SingleCluster.apt.vm
@@ -11,12 +11,12 @@
 ~~ limitations under the License. See accompanying LICENSE file.
 
   ---
-  Hadoop MapReduce Next Generation ${project.version} - Setting up a Single Node Cluster.
+  Hadoop ${project.version} - Setting up a Single Node Cluster.
   ---
   ---
   ${maven.build.timestamp}
 
-Hadoop MapReduce Next Generation - Setting up a Single Node Cluster.
+Hadoop - Setting up a Single Node Cluster.
 
 %{toc|section=1|fromDepth=0}
 
@@ -46,7 +46,9 @@ Hadoop MapReduce Next Generation - Setting up a Single Node Cluster.
          HadoopJavaVersions}}.
 
    [[2]] ssh must be installed and sshd must be running to use the Hadoop
-         scripts that manage remote Hadoop daemons.
+         scripts that manage remote Hadoop daemons if the optional start
+         and stop scripts are to be used.  Additionally, it is recommmended that
+         pdsh also be installed for better ssh resource management.
 
 ** Installing Software
 
@@ -57,7 +59,7 @@ Hadoop MapReduce Next Generation - Setting up a Single Node Cluster.
 
 ----
   $ sudo apt-get install ssh
-  $ sudo apt-get install rsync
+  $ sudo apt-get install pdsh
 ----
 
 * Download
@@ -75,9 +77,6 @@ Hadoop MapReduce Next Generation - Setting up a Single Node Cluster.
 ----
   # set to the root of your Java installation
   export JAVA_HOME=/usr/java/latest
-
-  # Assuming your installation directory is /usr/local/hadoop
-  export HADOOP_PREFIX=/usr/local/hadoop
 ----
 
   Try the following command:
@@ -158,6 +157,7 @@ Hadoop MapReduce Next Generation - Setting up a Single Node Cluster.
 ----
   $ ssh-keygen -t dsa -P '' -f ~/.ssh/id_dsa
   $ cat ~/.ssh/id_dsa.pub >> ~/.ssh/authorized_keys
+  $ chmod 0700 ~/.ssh/authorized_keys
 ----
 
 ** Execution
@@ -228,7 +228,7 @@ Hadoop MapReduce Next Generation - Setting up a Single Node Cluster.
   $ sbin/stop-dfs.sh
 ----
 
-** YARN on Single Node
+** YARN on a Single Node
 
   You can run a MapReduce job on YARN in a pseudo-distributed mode by setting
   a few parameters and running ResourceManager daemon and NodeManager daemon
@@ -239,7 +239,7 @@ Hadoop MapReduce Next Generation - Setting up a Single Node Cluster.
 
   [[1]] Configure parameters as follows:
 
-        etc/hadoop/mapred-site.xml:
+        <<<etc/hadoop/mapred-site.xml>>>:
 
 +---+
 <configuration>
@@ -250,7 +250,7 @@ Hadoop MapReduce Next Generation - Setting up a Single Node Cluster.
 </configuration>
 +---+
 
-        etc/hadoop/yarn-site.xml:
+        <<<etc/hadoop/yarn-site.xml>>>:
 
 +---+
 <configuration>


[04/18] hadoop git commit: MAPREDUCE-6149. Document override log4j.properties in MR job. Contributed by Junping Du.

Posted by zh...@apache.org.
MAPREDUCE-6149. Document override log4j.properties in MR job. Contributed by Junping Du.


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

Branch: refs/heads/HDFS-EC
Commit: 5abc670b76a11a2e1edff732822b7edfe09c1278
Parents: f123b71
Author: Harsh J <ha...@cloudera.com>
Authored: Tue Dec 30 23:12:32 2014 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:36 2015 -0800

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                 |  3 +++
 .../src/main/resources/mapred-default.xml            | 15 +++++++++++++++
 2 files changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5abc670b/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 0a9ee8d..82295de 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -238,6 +238,9 @@ Release 2.7.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-6149. Document override log4j.properties in MR job.
+    (Junping Du via harsh)
+
     MAPREDUCE-6194. Bubble up final exception in failures during creation
     of output collectors (Varun Saxena via harsh)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5abc670b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index 6e0deaa..30e291b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -324,6 +324,8 @@
   <value>INFO</value>
   <description>The logging level for the map task. The allowed levels are:
   OFF, FATAL, ERROR, WARN, INFO, DEBUG, TRACE and ALL.
+  The setting here could be overridden if "mapreduce.job.log4j-properties-file"
+  is set.
   </description>
 </property>
 
@@ -332,6 +334,8 @@
   <value>INFO</value>
   <description>The logging level for the reduce task. The allowed levels are:
   OFF, FATAL, ERROR, WARN, INFO, DEBUG, TRACE and ALL.
+  The setting here could be overridden if "mapreduce.job.log4j-properties-file"
+  is set.
   </description>
 </property>
 
@@ -1163,6 +1167,17 @@
   </description>
 </property>
 
+  <property>
+    <name>mapreduce.job.log4j-properties-file</name>
+    <value></value>
+    <description>Used to override the default settings of log4j in container-log4j.properties
+    for NodeManager. Like container-log4j.properties, it requires certain
+    framework appenders properly defined in this overriden file. The file on the
+    path will be added to distributed cache and classpath. If no-scheme is given
+    in the path, it defaults to point to a log4j file on the local FS.
+    </description>
+  </property>
+
 <property>
   <name>mapreduce.job.end-notification.max.retry.interval</name>
   <value>5000</value>


[10/18] hadoop git commit: HDFS-7563. NFS gateway parseStaticMap NumberFormatException. Contributed by Yongjun Zhang

Posted by zh...@apache.org.
HDFS-7563. NFS gateway parseStaticMap NumberFormatException. Contributed by Yongjun Zhang


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

Branch: refs/heads/HDFS-EC
Commit: de378cb59f00234839009b66eebc5b3b0040c30f
Parents: 1e28fa6
Author: Brandon Li <br...@apache.org>
Authored: Fri Jan 2 10:49:50 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:37 2015 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/security/ShellBasedIdMapping.java | 4 ++--
 .../org/apache/hadoop/security/TestShellBasedIdMapping.java  | 8 +++++++-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                  | 3 +++
 3 files changed, 12 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/de378cb5/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java
index e995cb6..428e835 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java
@@ -578,8 +578,8 @@ public class ShellBasedIdMapping implements IdMappingServiceProvider {
         // We know the line is fine to parse without error checking like this
         // since it matched the regex above.
         String firstComponent = lineMatcher.group(1);
-        int remoteId = Integer.parseInt(lineMatcher.group(2));
-        int localId = Integer.parseInt(lineMatcher.group(3));
+        int remoteId = parseId(lineMatcher.group(2));
+        int localId = parseId(lineMatcher.group(3));
         if (firstComponent.equals("uid")) {
           uidMapping.put(localId, remoteId);
         } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de378cb5/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedIdMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedIdMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedIdMapping.java
index ec8ac1d..857c706 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedIdMapping.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestShellBasedIdMapping.java
@@ -55,7 +55,9 @@ public class TestShellBasedIdMapping {
         "uid 13 302\n" +
         "gid\t11\t201\n" + // Tabs instead of spaces.
         "\n" + // Entirely empty line.
-        "gid 12 202";
+        "gid 12 202\n" +
+        "uid 4294967294 123\n" +
+        "gid 4294967295 321";
     OutputStream out = new FileOutputStream(tempStaticMapFile);
     out.write(staticMapFileContents.getBytes());
     out.close();
@@ -71,6 +73,10 @@ public class TestShellBasedIdMapping {
     assertEquals(10000, (int)parsedMap.uidMapping.get(10001));
     // Ensure pass-through of unmapped IDs works.
     assertEquals(1000, (int)parsedMap.uidMapping.get(1000));
+    
+    assertEquals(-2, (int)parsedMap.uidMapping.get(123));
+    assertEquals(-1, (int)parsedMap.gidMapping.get(321));
+    
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de378cb5/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 8525402..8117383 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -636,6 +636,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7456. De-duplicate AclFeature instances with same AclEntries do reduce
     memory footprint of NameNode (vinayakumarb)
 
+    HDFS-7563. NFS gateway parseStaticMap NumberFormatException 
+    (Yongjun Zhang via brandonli)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[11/18] hadoop git commit: YARN-2991. Fixed DrainDispatcher to reuse the draining code path in AsyncDispatcher. Contributed by Rohith Sharmaks.

Posted by zh...@apache.org.
YARN-2991. Fixed DrainDispatcher to reuse the draining code path in AsyncDispatcher. Contributed by Rohith Sharmaks.


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

Branch: refs/heads/HDFS-EC
Commit: 10415a06014af60ee3a97dffab1cb79d71a365a7
Parents: 80f386d
Author: Zhijie Shen <zj...@apache.org>
Authored: Fri Jan 2 11:08:47 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:37 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../hadoop/yarn/event/AsyncDispatcher.java      |  7 +++
 .../hadoop/yarn/event/DrainDispatcher.java      | 50 +-------------------
 3 files changed, 11 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/10415a06/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 01df44f..e6694f1 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -305,6 +305,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2987. Fixed ClientRMService#getQueueInfo to check against queue and
     app ACLs. (Varun Saxena via jianhe)
 
+    YARN-2991. Fixed DrainDispatcher to reuse the draining code path in
+    AsyncDispatcher. (Rohith Sharmaks via zjshen)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10415a06/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
index 28be6ac..d36d841 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
@@ -34,6 +34,8 @@ import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Dispatches {@link Event}s in a separate thread. Currently only single thread
  * does that. Potentially there could be multiple channels for each event type
@@ -282,4 +284,9 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
       }
     };
   }
+
+  @VisibleForTesting
+  protected boolean isDrained() {
+    return this.drained;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10415a06/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
index 803b2bb..da5ae44 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
@@ -23,68 +23,20 @@ import java.util.concurrent.LinkedBlockingQueue;
 @SuppressWarnings("rawtypes")
 public class DrainDispatcher extends AsyncDispatcher {
 
-// flagrant initialize abuse throughout, but safe per
-// http://java.sun.com/docs/books/jls/third_edition/html/typesValues.html#96595
-// and similar grotesqueries
-  private volatile boolean drained = false;
-  private final BlockingQueue<Event> queue;
-  final Object mutex;
-
   public DrainDispatcher() {
     this(new LinkedBlockingQueue<Event>());
   }
 
   private DrainDispatcher(BlockingQueue<Event> eventQueue) {
     super(eventQueue);
-    this.queue = eventQueue;
-    this.mutex = this;
   }
 
   /**
    * Busy loop waiting for all queued events to drain.
    */
   public void await() {
-    while (!drained) {
+    while (!isDrained()) {
       Thread.yield();
     }
   }
-
-  @Override
-  Runnable createThread() {
-    return new Runnable() {
-      @Override
-      public void run() {
-        while (!Thread.currentThread().isInterrupted()) {
-          synchronized (mutex) {
-            // !drained if dispatch queued new events on this dispatcher
-            drained = queue.isEmpty();
-          }
-          Event event;
-          try {
-            event = queue.take();
-          } catch(InterruptedException ie) {
-            return;
-          }
-          if (event != null) {
-            dispatch(event);
-          }
-        }
-      }
-    };
-  }
-
-  @Override
-  public EventHandler getEventHandler() {
-    final EventHandler actual = super.getEventHandler();
-    return new EventHandler() {
-      @Override
-      public void handle(Event event) {
-        synchronized (mutex) {
-          actual.handle(event);
-          drained = false;
-        }
-      }
-    };
-  }
-
 }


[07/18] hadoop git commit: YARN-2998. Abstract out scheduler independent PlanFollower components. (Anubhav Dhoot via kasha)

Posted by zh...@apache.org.
YARN-2998. Abstract out scheduler independent PlanFollower components. (Anubhav Dhoot via kasha)


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

Branch: refs/heads/HDFS-EC
Commit: 1e28fa622d02863996901beb1ebc75b8270cab3d
Parents: 82add25
Author: Karthik Kambatla <ka...@apache.org>
Authored: Tue Dec 30 19:52:43 2014 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:37 2015 -0800

----------------------------------------------------------------------
 .../sls/scheduler/ResourceSchedulerWrapper.java |   6 +
 hadoop-yarn-project/CHANGES.txt                 |   5 +
 .../AbstractSchedulerPlanFollower.java          | 412 +++++++++++++++++++
 .../CapacitySchedulerPlanFollower.java          | 361 ++++------------
 .../scheduler/YarnScheduler.java                |   5 +
 .../scheduler/fair/FSLeafQueue.java             |   4 +-
 .../scheduler/fair/FairScheduler.java           |   3 +-
 .../scheduler/fifo/FifoScheduler.java           |   5 +
 .../reservation/ReservationSystemTestUtil.java  |   4 +-
 .../TestCapacitySchedulerPlanFollower.java      | 209 +++-------
 .../TestSchedulerPlanFollowerBase.java          | 191 +++++++++
 11 files changed, 755 insertions(+), 450 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e28fa62/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
index a65f776..0a80291 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
@@ -84,6 +84,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoSchedule
 import org.apache.hadoop.yarn.sls.SLSRunner;
 import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
 import org.apache.hadoop.yarn.sls.web.SLSWebApp;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.log4j.Logger;
 
@@ -867,6 +868,11 @@ final public class ResourceSchedulerWrapper
   }
 
   @Override
+  public ResourceCalculator getResourceCalculator() {
+    return scheduler.getResourceCalculator();
+  }
+
+  @Override
   public int getNumClusterNodes() {
     return scheduler.getNumClusterNodes();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e28fa62/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f2115e1..01df44f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -70,6 +70,7 @@ Release 2.7.0 - UNRELEASED
     YARN-2738. [YARN-2574] Add FairReservationSystem for FairScheduler. 
     (Anubhav Dhoot via kasha)
 
+
   IMPROVEMENTS
 
     YARN-2950. Change message to mandate, not suggest JS requirement on UI.
@@ -157,6 +158,10 @@ Release 2.7.0 - UNRELEASED
 
     YARN-2943. Added node-labels page on RM web UI. (Wangda Tan via jianhe)
 
+    YARN-2998. Abstract out scheduler independent PlanFollower components. 
+    (Anubhav Dhoot via kasha)
+
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e28fa62/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.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/reservation/AbstractSchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
new file mode 100644
index 0000000..0de4dcf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
@@ -0,0 +1,412 @@
+/**
+ * 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.reservation;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.PlanQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
+
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(CapacitySchedulerPlanFollower.class);
+
+  protected Collection<Plan> plans = new ArrayList<Plan>();
+  protected YarnScheduler scheduler;
+  protected Clock clock;
+
+  @Override
+  public void init(Clock clock, ResourceScheduler sched, Collection<Plan> plans) {
+    this.clock = clock;
+    this.scheduler = sched;
+    this.plans.addAll(plans);
+  }
+
+  @Override
+  public synchronized void run() {
+    for (Plan plan : plans) {
+      synchronizePlan(plan);
+    }
+  }
+
+  @Override
+  public synchronized void setPlans(Collection<Plan> plans) {
+    this.plans.clear();
+    this.plans.addAll(plans);
+  }
+
+  @Override
+  public synchronized void synchronizePlan(Plan plan) {
+     String planQueueName = plan.getQueueName();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Running plan follower edit policy for plan: " + planQueueName);
+    }
+    // align with plan step
+    long step = plan.getStep();
+    long now = clock.getTime();
+    if (now % step != 0) {
+      now += step - (now % step);
+    }
+    Queue planQueue = getPlanQueue(planQueueName);
+    if (planQueue == null) return;
+
+    // first we publish to the plan the current availability of resources
+    Resource clusterResources = scheduler.getClusterResource();
+    Resource planResources = getPlanResources(plan, planQueue,
+        clusterResources);
+
+    Set<ReservationAllocation> currentReservations =
+        plan.getReservationsAtTime(now);
+    Set<String> curReservationNames = new HashSet<String>();
+    Resource reservedResources = Resource.newInstance(0, 0);
+    int numRes = getReservedResources(now, currentReservations,
+        curReservationNames, reservedResources);
+
+    // create the default reservation queue if it doesnt exist
+    String defReservationId = getReservationIdFromQueueName(planQueueName) +
+        PlanQueue.DEFAULT_QUEUE_SUFFIX;
+    String defReservationQueue = getReservationQueueName(planQueueName,
+        defReservationId);
+    createDefaultReservationQueue(planQueueName, planQueue,
+        defReservationId);
+    curReservationNames.add(defReservationId);
+
+    // if the resources dedicated to this plan has shrunk invoke replanner
+    if (arePlanResourcesLessThanReservations(clusterResources, planResources,
+        reservedResources)) {
+      try {
+        plan.getReplanner().plan(plan, null);
+      } catch (PlanningException e) {
+        LOG.warn("Exception while trying to replan: {}", planQueueName, e);
+      }
+    }
+    // identify the reservations that have expired and new reservations that
+    // have to be activated
+    List<? extends Queue> resQueues = getChildReservationQueues(planQueue);
+    Set<String> expired = new HashSet<String>();
+    for (Queue resQueue : resQueues) {
+      String resQueueName = resQueue.getQueueName();
+      String reservationId = getReservationIdFromQueueName(resQueueName);
+      if (curReservationNames.contains(reservationId)) {
+        // it is already existing reservation, so needed not create new
+        // reservation queue
+        curReservationNames.remove(reservationId);
+      } else {
+        // the reservation has termination, mark for cleanup
+        expired.add(reservationId);
+      }
+    }
+    // garbage collect expired reservations
+    cleanupExpiredQueues(planQueueName, plan.getMoveOnExpiry(), expired,
+        defReservationQueue);
+
+    // Add new reservations and update existing ones
+    float totalAssignedCapacity = 0f;
+    if (currentReservations != null) {
+      // first release all excess capacity in default queue
+      try {
+        setQueueEntitlement(planQueueName, defReservationQueue, 0f, 1.0f);
+      } catch (YarnException e) {
+        LOG.warn(
+            "Exception while trying to release default queue capacity for plan: {}",
+            planQueueName, e);
+      }
+      // sort allocations from the one giving up the most resources, to the
+      // one asking for the most
+      // avoid order-of-operation errors that temporarily violate 100%
+      // capacity bound
+      List<ReservationAllocation> sortedAllocations =
+          sortByDelta(
+              new ArrayList<ReservationAllocation>(currentReservations), now,
+              plan);
+      for (ReservationAllocation res : sortedAllocations) {
+        String currResId = res.getReservationId().toString();
+        if (curReservationNames.contains(currResId)) {
+          addReservationQueue(planQueueName, planQueue, currResId);
+        }
+        Resource capToAssign = res.getResourcesAtTime(now);
+        float targetCapacity = 0f;
+        if (planResources.getMemory() > 0
+            && planResources.getVirtualCores() > 0) {
+          targetCapacity =
+              calculateReservationToPlanRatio(clusterResources,
+                  planResources,
+                  capToAssign);
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(
+              "Assigning capacity of {} to queue {} with target capacity {}",
+              capToAssign, currResId, targetCapacity);
+        }
+        // set maxCapacity to 100% unless the job requires gang, in which
+        // case we stick to capacity (as running early/before is likely a
+        // waste of resources)
+        float maxCapacity = 1.0f;
+        if (res.containsGangs()) {
+          maxCapacity = targetCapacity;
+        }
+        try {
+          setQueueEntitlement(planQueueName, currResId, targetCapacity, maxCapacity);
+        } catch (YarnException e) {
+          LOG.warn("Exception while trying to size reservation for plan: {}",
+              currResId, planQueueName, e);
+        }
+        totalAssignedCapacity += targetCapacity;
+      }
+    }
+    // compute the default queue capacity
+    float defQCap = 1.0f - totalAssignedCapacity;
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("PlanFollowerEditPolicyTask: total Plan Capacity: {} "
+          + "currReservation: {} default-queue capacity: {}", planResources,
+          numRes, defQCap);
+    }
+    // set the default queue to eat-up all remaining capacity
+    try {
+      setQueueEntitlement(planQueueName, defReservationQueue, defQCap, 1.0f);
+    } catch (YarnException e) {
+      LOG.warn(
+          "Exception while trying to reclaim default queue capacity for plan: {}",
+          planQueueName, e);
+    }
+    // garbage collect finished reservations from plan
+    try {
+      plan.archiveCompletedReservations(now);
+    } catch (PlanningException e) {
+      LOG.error("Exception in archiving completed reservations: ", e);
+    }
+    LOG.info("Finished iteration of plan follower edit policy for plan: "
+        + planQueueName);
+
+    // Extension: update plan with app states,
+    // useful to support smart replanning
+  }
+
+  protected String getReservationIdFromQueueName(String resQueueName) {
+    return resQueueName;
+  }
+
+  protected void setQueueEntitlement(String planQueueName, String currResId,
+      float targetCapacity,
+      float maxCapacity) throws YarnException {
+    String reservationQueueName = getReservationQueueName(planQueueName,
+        currResId);
+    scheduler.setEntitlement(reservationQueueName, new QueueEntitlement(
+        targetCapacity, maxCapacity));
+  }
+
+  // Schedulers have different ways of naming queues. See YARN-2773
+  protected String getReservationQueueName(String planQueueName,
+      String reservationId) {
+    return reservationId;
+  }
+
+  /**
+   * First sets entitlement of queues to zero to prevent new app submission.
+   * Then move all apps in the set of queues to the parent plan queue's default
+   * reservation queue if move is enabled. Finally cleanups the queue by killing
+   * any apps (if move is disabled or move failed) and removing the queue
+   */
+  protected void cleanupExpiredQueues(String planQueueName,
+      boolean shouldMove, Set<String> toRemove, String defReservationQueue) {
+    for (String expiredReservationId : toRemove) {
+      try {
+        // reduce entitlement to 0
+        String expiredReservation = getReservationQueueName(planQueueName,
+            expiredReservationId);
+        setQueueEntitlement(planQueueName, expiredReservation, 0.0f, 0.0f);
+        if (shouldMove) {
+          moveAppsInQueueSync(expiredReservation, defReservationQueue);
+        }
+        if (scheduler.getAppsInQueue(expiredReservation).size() > 0) {
+          scheduler.killAllAppsInQueue(expiredReservation);
+          LOG.info("Killing applications in queue: {}", expiredReservation);
+        } else {
+          scheduler.removeQueue(expiredReservation);
+          LOG.info("Queue: " + expiredReservation + " removed");
+        }
+      } catch (YarnException e) {
+        LOG.warn("Exception while trying to expire reservation: {}",
+            expiredReservationId, e);
+      }
+    }
+  }
+
+  /**
+   * Move all apps in the set of queues to the parent plan queue's default
+   * reservation queue in a synchronous fashion
+   */
+  private void moveAppsInQueueSync(String expiredReservation,
+                                   String defReservationQueue) {
+    List<ApplicationAttemptId> activeApps =
+        scheduler.getAppsInQueue(expiredReservation);
+    if (activeApps.isEmpty()) {
+      return;
+    }
+    for (ApplicationAttemptId app : activeApps) {
+      // fallback to parent's default queue
+      try {
+        scheduler.moveApplication(app.getApplicationId(), defReservationQueue);
+      } catch (YarnException e) {
+        LOG.warn(
+            "Encountered unexpected error during migration of application: {}" +
+                " from reservation: {}",
+            app, expiredReservation, e);
+      }
+    }
+  }
+
+  protected int getReservedResources(long now, Set<ReservationAllocation>
+      currentReservations, Set<String> curReservationNames,
+                                     Resource reservedResources) {
+    int numRes = 0;
+    if (currentReservations != null) {
+      numRes = currentReservations.size();
+      for (ReservationAllocation reservation : currentReservations) {
+        curReservationNames.add(reservation.getReservationId().toString());
+        Resources.addTo(reservedResources, reservation.getResourcesAtTime(now));
+      }
+    }
+    return numRes;
+  }
+
+  /**
+   * Sort in the order from the least new amount of resources asked (likely
+   * negative) to the highest. This prevents "order-of-operation" errors related
+   * to exceeding 100% capacity temporarily.
+   */
+  protected List<ReservationAllocation> sortByDelta(
+      List<ReservationAllocation> currentReservations, long now, Plan plan) {
+    Collections.sort(currentReservations, new ReservationAllocationComparator(
+        now, this, plan));
+    return currentReservations;
+  }
+
+  /**
+   * Get queue associated with reservable queue named
+   * @param planQueueName Name of the reservable queue
+   * @return queue associated with the reservable queue
+   */
+  protected abstract Queue getPlanQueue(String planQueueName);
+
+  /**
+   * Calculates ratio of reservationResources to planResources
+   */
+  protected abstract float calculateReservationToPlanRatio(
+      Resource clusterResources, Resource planResources,
+      Resource reservationResources);
+
+  /**
+   * Check if plan resources are less than expected reservation resources
+   */
+  protected abstract boolean arePlanResourcesLessThanReservations(
+      Resource clusterResources, Resource planResources,
+      Resource reservedResources);
+
+  /**
+   * Get a list of reservation queues for this planQueue
+   */
+  protected abstract List<? extends Queue> getChildReservationQueues(
+      Queue planQueue);
+
+  /**
+   * Add a new reservation queue for reservation currResId for this planQueue
+   */
+  protected abstract void addReservationQueue(
+      String planQueueName, Queue queue, String currResId);
+
+  /**
+   * Creates the default reservation queue for use when no reservation is
+   * used for applications submitted to this planQueue
+   */
+  protected abstract void createDefaultReservationQueue(
+      String planQueueName, Queue queue, String defReservationQueue);
+
+  /**
+   * Get plan resources for this planQueue
+   */
+  protected abstract Resource getPlanResources(
+      Plan plan, Queue queue, Resource clusterResources);
+
+  /**
+   * Get reservation queue resources if it exists otherwise return null
+   */
+  protected abstract Resource getReservationQueueResourceIfExists(Plan plan,
+      ReservationId reservationId);
+
+  private static class ReservationAllocationComparator implements
+      Comparator<ReservationAllocation> {
+    AbstractSchedulerPlanFollower planFollower;
+    long now;
+    Plan plan;
+
+    ReservationAllocationComparator(long now,
+        AbstractSchedulerPlanFollower planFollower, Plan plan) {
+      this.now = now;
+      this.planFollower = planFollower;
+      this.plan = plan;
+    }
+
+    private Resource getUnallocatedReservedResources(
+        ReservationAllocation reservation) {
+      Resource resResource;
+      Resource reservationResource = planFollower
+          .getReservationQueueResourceIfExists
+              (plan, reservation.getReservationId());
+      if (reservationResource != null) {
+        resResource =
+            Resources.subtract(
+                reservation.getResourcesAtTime(now),
+                reservationResource);
+      } else {
+        resResource = reservation.getResourcesAtTime(now);
+      }
+      return resResource;
+    }
+
+    @Override
+    public int compare(ReservationAllocation lhs, ReservationAllocation rhs) {
+      // compute delta between current and previous reservation, and compare
+      // based on that
+      Resource lhsRes = getUnallocatedReservedResources(lhs);
+      Resource rhsRes = getUnallocatedReservedResources(rhs);
+      return lhsRes.compareTo(rhsRes);
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e28fa62/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.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/reservation/CapacitySchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java
index 126560a..61772c9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/CapacitySchedulerPlanFollower.java
@@ -19,26 +19,19 @@
 package org.apache.hadoop.yarn.server.resourcemanager.reservation;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
 
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
 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.capacity.PlanQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ReservationQueue;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.slf4j.Logger;
@@ -58,319 +51,119 @@ import org.slf4j.LoggerFactory;
  * differences among existing queues). This makes it resilient to frequency of
  * synchronization, and RM restart issues (no "catch up" is necessary).
  */
-public class CapacitySchedulerPlanFollower implements PlanFollower {
+public class CapacitySchedulerPlanFollower extends AbstractSchedulerPlanFollower {
 
   private static final Logger LOG = LoggerFactory
       .getLogger(CapacitySchedulerPlanFollower.class);
 
-  private Collection<Plan> plans = new ArrayList<Plan>();
-
-  private Clock clock;
-  private CapacityScheduler scheduler;
+  private CapacityScheduler cs;
 
   @Override
   public void init(Clock clock, ResourceScheduler sched, Collection<Plan> plans) {
+    super.init(clock, sched, plans);
     LOG.info("Initializing Plan Follower Policy:"
         + this.getClass().getCanonicalName());
     if (!(sched instanceof CapacityScheduler)) {
       throw new YarnRuntimeException(
           "CapacitySchedulerPlanFollower can only work with CapacityScheduler");
     }
-    this.clock = clock;
-    this.scheduler = (CapacityScheduler) sched;
-    this.plans.addAll(plans);
+    this.cs = (CapacityScheduler) sched;
   }
 
   @Override
-  public synchronized void run() {
-    for (Plan plan : plans) {
-      synchronizePlan(plan);
+  protected Queue getPlanQueue(String planQueueName) {
+    CSQueue queue = cs.getQueue(planQueueName);
+    if (!(queue instanceof PlanQueue)) {
+      LOG.error("The Plan is not an PlanQueue!");
+      return null;
     }
+    return queue;
   }
 
   @Override
-  public synchronized void synchronizePlan(Plan plan) {
-    String planQueueName = plan.getQueueName();
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Running plan follower edit policy for plan: " + planQueueName);
-    }
-    // align with plan step
-    long step = plan.getStep();
-    long now = clock.getTime();
-    if (now % step != 0) {
-      now += step - (now % step);
-    }
-    CSQueue queue = scheduler.getQueue(planQueueName);
-    if (!(queue instanceof PlanQueue)) {
-      LOG.error("The Plan is not an PlanQueue!");
-      return;
-    }
-    PlanQueue planQueue = (PlanQueue) queue;
-    // first we publish to the plan the current availability of resources
-    Resource clusterResources = scheduler.getClusterResource();
-    float planAbsCap = planQueue.getAbsoluteCapacity();
-    Resource planResources = Resources.multiply(clusterResources, planAbsCap);
-    plan.setTotalCapacity(planResources);
+  protected float calculateReservationToPlanRatio(
+      Resource clusterResources, Resource planResources,
+      Resource reservationResources) {
+    return Resources.divide(cs.getResourceCalculator(),
+        clusterResources, reservationResources, planResources);
+  }
 
-    Set<ReservationAllocation> currentReservations =
-        plan.getReservationsAtTime(now);
-    Set<String> curReservationNames = new HashSet<String>();
-    Resource reservedResources = Resource.newInstance(0, 0);
-    int numRes = 0;
-    if (currentReservations != null) {
-      numRes = currentReservations.size();
-      for (ReservationAllocation reservation : currentReservations) {
-        curReservationNames.add(reservation.getReservationId().toString());
-        Resources.addTo(reservedResources, reservation.getResourcesAtTime(now));
-      }
+  @Override
+  protected boolean arePlanResourcesLessThanReservations(
+      Resource clusterResources, Resource planResources,
+      Resource reservedResources) {
+    return Resources.greaterThan(cs.getResourceCalculator(),
+        clusterResources, reservedResources, planResources);
+  }
+
+  @Override
+  protected List<? extends Queue> getChildReservationQueues(Queue queue) {
+    PlanQueue planQueue = (PlanQueue)queue;
+    List<CSQueue> childQueues = planQueue.getChildQueues();
+    return childQueues;
+  }
+
+  @Override
+  protected void addReservationQueue(
+      String planQueueName, Queue queue, String currResId) {
+    PlanQueue planQueue = (PlanQueue)queue;
+    try {
+      ReservationQueue resQueue =
+          new ReservationQueue(cs, currResId, planQueue);
+      cs.addQueue(resQueue);
+    } catch (SchedulerDynamicEditException e) {
+      LOG.warn(
+          "Exception while trying to activate reservation: {} for plan: {}",
+          currResId, planQueueName, e);
+    } catch (IOException e) {
+      LOG.warn(
+          "Exception while trying to activate reservation: {} for plan: {}",
+          currResId, planQueueName, e);
     }
-    // create the default reservation queue if it doesnt exist
-    String defReservationQueue = planQueueName + PlanQueue.DEFAULT_QUEUE_SUFFIX;
-    if (scheduler.getQueue(defReservationQueue) == null) {
+  }
+
+  @Override
+  protected void createDefaultReservationQueue(
+      String planQueueName, Queue queue, String defReservationId) {
+    PlanQueue planQueue = (PlanQueue)queue;
+    if (cs.getQueue(defReservationId) == null) {
       try {
         ReservationQueue defQueue =
-            new ReservationQueue(scheduler, defReservationQueue, planQueue);
-        scheduler.addQueue(defQueue);
+            new ReservationQueue(cs, defReservationId, planQueue);
+        cs.addQueue(defQueue);
       } catch (SchedulerDynamicEditException e) {
         LOG.warn(
             "Exception while trying to create default reservation queue for plan: {}",
             planQueueName, e);
       } catch (IOException e) {
         LOG.warn(
-            "Exception while trying to create default reservation queue for plan: {}",
-            planQueueName, e);
-      }
-    }
-    curReservationNames.add(defReservationQueue);
-    // if the resources dedicated to this plan has shrunk invoke replanner
-    if (Resources.greaterThan(scheduler.getResourceCalculator(),
-        clusterResources, reservedResources, planResources)) {
-      try {
-        plan.getReplanner().plan(plan, null);
-      } catch (PlanningException e) {
-        LOG.warn("Exception while trying to replan: {}", planQueueName, e);
-      }
-    }
-    // identify the reservations that have expired and new reservations that
-    // have to be activated
-    List<CSQueue> resQueues = planQueue.getChildQueues();
-    Set<String> expired = new HashSet<String>();
-    for (CSQueue resQueue : resQueues) {
-      String resQueueName = resQueue.getQueueName();
-      if (curReservationNames.contains(resQueueName)) {
-        // it is already existing reservation, so needed not create new
-        // reservation queue
-        curReservationNames.remove(resQueueName);
-      } else {
-        // the reservation has termination, mark for cleanup
-        expired.add(resQueueName);
-      }
-    }
-    // garbage collect expired reservations
-    cleanupExpiredQueues(plan.getMoveOnExpiry(), expired, defReservationQueue);
-
-    // Add new reservations and update existing ones
-    float totalAssignedCapacity = 0f;
-    if (currentReservations != null) {
-      // first release all excess capacity in default queue
-      try {
-        scheduler.setEntitlement(defReservationQueue, new QueueEntitlement(0f,
-            1.0f));
-      } catch (YarnException e) {
-        LOG.warn(
-            "Exception while trying to release default queue capacity for plan: {}",
+            "Exception while trying to create default reservation queue for " +
+                "plan: {}",
             planQueueName, e);
       }
-      // sort allocations from the one giving up the most resources, to the
-      // one asking for the most
-      // avoid order-of-operation errors that temporarily violate 100%
-      // capacity bound
-      List<ReservationAllocation> sortedAllocations =
-          sortByDelta(
-              new ArrayList<ReservationAllocation>(currentReservations), now);
-      for (ReservationAllocation res : sortedAllocations) {
-        String currResId = res.getReservationId().toString();
-        if (curReservationNames.contains(currResId)) {
-          try {
-            ReservationQueue resQueue =
-                new ReservationQueue(scheduler, currResId, planQueue);
-            scheduler.addQueue(resQueue);
-          } catch (SchedulerDynamicEditException e) {
-            LOG.warn(
-                "Exception while trying to activate reservation: {} for plan: {}",
-                currResId, planQueueName, e);
-          } catch (IOException e) {
-            LOG.warn(
-                "Exception while trying to activate reservation: {} for plan: {}",
-                currResId, planQueueName, e);
-          }
-        }
-        Resource capToAssign = res.getResourcesAtTime(now);
-        float targetCapacity = 0f;
-        if (planResources.getMemory() > 0
-            && planResources.getVirtualCores() > 0) {
-          targetCapacity =
-              Resources.divide(scheduler.getResourceCalculator(),
-                  clusterResources, capToAssign, planResources);
-        }
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(
-              "Assigning capacity of {} to queue {} with target capacity {}",
-              capToAssign, currResId, targetCapacity);
-        }
-        // set maxCapacity to 100% unless the job requires gang, in which
-        // case we stick to capacity (as running early/before is likely a
-        // waste of resources)
-        float maxCapacity = 1.0f;
-        if (res.containsGangs()) {
-          maxCapacity = targetCapacity;
-        }
-        try {
-          scheduler.setEntitlement(currResId, new QueueEntitlement(
-              targetCapacity, maxCapacity));
-        } catch (YarnException e) {
-          LOG.warn("Exception while trying to size reservation for plan: {}",
-              currResId, planQueueName, e);
-        }
-        totalAssignedCapacity += targetCapacity;
-      }
-    }
-    // compute the default queue capacity
-    float defQCap = 1.0f - totalAssignedCapacity;
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("PlanFollowerEditPolicyTask: total Plan Capacity: {} "
-          + "currReservation: {} default-queue capacity: {}", planResources,
-          numRes, defQCap);
-    }
-    // set the default queue to eat-up all remaining capacity
-    try {
-      scheduler.setEntitlement(defReservationQueue, new QueueEntitlement(
-          defQCap, 1.0f));
-    } catch (YarnException e) {
-      LOG.warn(
-          "Exception while trying to reclaim default queue capacity for plan: {}",
-          planQueueName, e);
-    }
-    // garbage collect finished reservations from plan
-    try {
-      plan.archiveCompletedReservations(now);
-    } catch (PlanningException e) {
-      LOG.error("Exception in archiving completed reservations: ", e);
-    }
-    LOG.info("Finished iteration of plan follower edit policy for plan: "
-        + planQueueName);
-
-    // Extension: update plan with app states,
-    // useful to support smart replanning
-  }
-
-  /**
-   * Move all apps in the set of queues to the parent plan queue's default
-   * reservation queue in a synchronous fashion
-   */
-  private void moveAppsInQueueSync(String expiredReservation,
-      String defReservationQueue) {
-    List<ApplicationAttemptId> activeApps =
-        scheduler.getAppsInQueue(expiredReservation);
-    if (activeApps.isEmpty()) {
-      return;
-    }
-    for (ApplicationAttemptId app : activeApps) {
-      // fallback to parent's default queue
-      try {
-        scheduler.moveApplication(app.getApplicationId(), defReservationQueue);
-      } catch (YarnException e) {
-        LOG.warn(
-            "Encountered unexpected error during migration of application: {} from reservation: {}",
-            app, expiredReservation, e);
-      }
-    }
-  }
-
-  /**
-   * First sets entitlement of queues to zero to prevent new app submission.
-   * Then move all apps in the set of queues to the parent plan queue's default
-   * reservation queue if move is enabled. Finally cleanups the queue by killing
-   * any apps (if move is disabled or move failed) and removing the queue
-   */
-  private void cleanupExpiredQueues(boolean shouldMove, Set<String> toRemove,
-      String defReservationQueue) {
-    for (String expiredReservation : toRemove) {
-      try {
-        // reduce entitlement to 0
-        scheduler.setEntitlement(expiredReservation, new QueueEntitlement(0.0f,
-            0.0f));
-        if (shouldMove) {
-          moveAppsInQueueSync(expiredReservation, defReservationQueue);
-        }
-        if (scheduler.getAppsInQueue(expiredReservation).size() > 0) {
-          scheduler.killAllAppsInQueue(expiredReservation);
-          LOG.info("Killing applications in queue: {}", expiredReservation);
-        } else {
-          scheduler.removeQueue(expiredReservation);
-          LOG.info("Queue: " + expiredReservation + " removed");
-        }
-      } catch (YarnException e) {
-        LOG.warn("Exception while trying to expire reservation: {}",
-            expiredReservation, e);
-      }
     }
   }
 
   @Override
-  public synchronized void setPlans(Collection<Plan> plans) {
-    this.plans.clear();
-    this.plans.addAll(plans);
-  }
-
-  /**
-   * Sort in the order from the least new amount of resources asked (likely
-   * negative) to the highest. This prevents "order-of-operation" errors related
-   * to exceeding 100% capacity temporarily.
-   */
-  private List<ReservationAllocation> sortByDelta(
-      List<ReservationAllocation> currentReservations, long now) {
-    Collections.sort(currentReservations, new ReservationAllocationComparator(
-        scheduler, now));
-    return currentReservations;
+  protected Resource getPlanResources(
+      Plan plan, Queue queue, Resource clusterResources) {
+    PlanQueue planQueue = (PlanQueue)queue;
+    float planAbsCap = planQueue.getAbsoluteCapacity();
+    Resource planResources = Resources.multiply(clusterResources, planAbsCap);
+    plan.setTotalCapacity(planResources);
+    return planResources;
   }
 
-  private static class ReservationAllocationComparator implements
-      Comparator<ReservationAllocation> {
-    CapacityScheduler scheduler;
-    long now;
-
-    ReservationAllocationComparator(CapacityScheduler scheduler, long now) {
-      this.scheduler = scheduler;
-      this.now = now;
-    }
-
-    private Resource getUnallocatedReservedResources(
-        ReservationAllocation reservation) {
-      Resource resResource;
-      CSQueue resQueue =
-          scheduler.getQueue(reservation.getReservationId().toString());
-      if (resQueue != null) {
-        resResource =
-            Resources.subtract(
-                reservation.getResourcesAtTime(now),
-                Resources.multiply(scheduler.getClusterResource(),
-                    resQueue.getAbsoluteCapacity()));
-      } else {
-        resResource = reservation.getResourcesAtTime(now);
-      }
-      return resResource;
-    }
-
-    @Override
-    public int compare(ReservationAllocation lhs, ReservationAllocation rhs) {
-      // compute delta between current and previous reservation, and compare
-      // based on that
-      Resource lhsRes = getUnallocatedReservedResources(lhs);
-      Resource rhsRes = getUnallocatedReservedResources(rhs);
-      return lhsRes.compareTo(rhsRes);
-    }
-
+  @Override
+  protected Resource getReservationQueueResourceIfExists(Plan plan,
+      ReservationId reservationId) {
+    CSQueue resQueue = cs.getQueue(reservationId.toString());
+    Resource reservationResource = null;
+    if (resQueue != null) {
+      reservationResource = Resources.multiply(cs.getClusterResource(),
+          resQueue.getAbsoluteCapacity());
+    }
+    return reservationResource;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e28fa62/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.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/YarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
index d1b5275..4a3a35c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 
 /**
  * This interface is used by the components to talk to the
@@ -98,6 +99,10 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
   @Stable
   public Resource getMaximumResourceCapability();
 
+  @LimitedPrivate("yarn")
+  @Evolving
+  ResourceCalculator getResourceCalculator();
+
   /**
    * Get the number of nodes available in the cluster.
    * @return the number of available nodes.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e28fa62/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 5862a73..91bea11 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
@@ -544,9 +544,9 @@ public class FSLeafQueue extends FSQueue {
   }
 
   private boolean isStarved(Resource share) {
-    Resource desiredShare = Resources.min(FairScheduler.getResourceCalculator(),
+    Resource desiredShare = Resources.min(scheduler.getResourceCalculator(),
         scheduler.getClusterResource(), share, getDemand());
-    return Resources.lessThan(FairScheduler.getResourceCalculator(),
+    return Resources.lessThan(scheduler.getResourceCalculator(),
         scheduler.getClusterResource(), getResourceUsage(), desiredShare);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e28fa62/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 49dfc3c..57b41af 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
@@ -1094,7 +1094,8 @@ public class FairScheduler extends
     return super.getApplicationAttempt(appAttemptId);
   }
 
-  public static ResourceCalculator getResourceCalculator() {
+  @Override
+  public ResourceCalculator getResourceCalculator() {
     return RESOURCE_CALCULATOR;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e28fa62/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 3d4c9dd..e006715 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
@@ -919,6 +919,11 @@ public class FifoScheduler extends
     return DEFAULT_QUEUE.getQueueUserAclInfo(null); 
   }
 
+  @Override
+  public ResourceCalculator getResourceCalculator() {
+    return resourceCalculator;
+  }
+
   private synchronized void addNode(RMNode nodeManager) {
     FiCaSchedulerNode schedulerNode = new FiCaSchedulerNode(nodeManager,
         usePortForNodeName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e28fa62/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
index d93af38..12c2583 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
@@ -104,7 +104,7 @@ public class ReservationSystemTestUtil {
         .assertTrue(newPlan.getSharingPolicy() instanceof CapacityOverTimePolicy);
   }
 
-  static void setupFSAllocationFile(String allocationFile)
+  public static void setupFSAllocationFile(String allocationFile)
       throws IOException {
     PrintWriter out = new PrintWriter(new FileWriter(allocationFile));
     out.println("<?xml version=\"1.0\"?>");
@@ -130,7 +130,7 @@ public class ReservationSystemTestUtil {
     out.close();
   }
 
-  static void updateFSAllocationFile(String allocationFile)
+  public static void updateFSAllocationFile(String allocationFile)
       throws IOException {
     PrintWriter out = new PrintWriter(new FileWriter(allocationFile));
     out.println("<?xml version=\"1.0\"?>");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e28fa62/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.java
index 4eedd42..c603f5b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.java
@@ -33,25 +33,20 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ReservationId;
-import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 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.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.PlanQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
-import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.After;
 import org.junit.Assert;
@@ -62,21 +57,12 @@ import org.junit.rules.TestName;
 import org.mockito.Matchers;
 import org.mockito.Mockito;
 
-public class TestCapacitySchedulerPlanFollower {
+public class TestCapacitySchedulerPlanFollower extends TestSchedulerPlanFollowerBase {
 
-  final static int GB = 1024;
-
-  private Clock mClock = null;
-  private CapacityScheduler scheduler = null;
   private RMContext rmContext;
   private RMContext spyRMContext;
   private CapacitySchedulerContext csContext;
-  private ReservationAgent mAgent;
-  private Plan plan;
-  private Resource minAlloc = Resource.newInstance(GB, 1);
-  private Resource maxAlloc = Resource.newInstance(GB * 8, 8);
-  private ResourceCalculator res = new DefaultResourceCalculator();
-  private CapacityOverTimePolicy policy = new CapacityOverTimePolicy();
+  private CapacityScheduler cs;
 
   @Rule
   public TestName name = new TestName();
@@ -84,7 +70,9 @@ public class TestCapacitySchedulerPlanFollower {
   @Before
   public void setUp() throws Exception {
     CapacityScheduler spyCs = new CapacityScheduler();
-    scheduler = spy(spyCs);
+    cs = spy(spyCs);
+    scheduler = cs;
+
     rmContext = TestUtils.getMockRMContext();
     spyRMContext = spy(rmContext);
 
@@ -100,7 +88,7 @@ public class TestCapacitySchedulerPlanFollower {
         new CapacitySchedulerConfiguration();
     ReservationSystemTestUtil.setupQueueConfiguration(csConf);
 
-    scheduler.setConf(csConf);
+    cs.setConf(csConf);
 
     csContext = mock(CapacitySchedulerContext.class);
     when(csContext.getConfiguration()).thenReturn(csConf);
@@ -119,9 +107,9 @@ public class TestCapacitySchedulerPlanFollower {
     when(csContext.getContainerTokenSecretManager()).thenReturn(
         containerTokenSecretManager);
 
-    scheduler.setRMContext(spyRMContext);
-    scheduler.init(csConf);
-    scheduler.start();
+    cs.setRMContext(spyRMContext);
+    cs.init(csConf);
+    cs.start();
 
     setupPlanFollower();
   }
@@ -132,7 +120,7 @@ public class TestCapacitySchedulerPlanFollower {
     mAgent = mock(ReservationAgent.class);
 
     String reservationQ = testUtil.getFullReservationQueueName();
-    CapacitySchedulerConfiguration csConf = scheduler.getConfiguration();
+    CapacitySchedulerConfiguration csConf = cs.getConfiguration();
     csConf.setReservationWindow(reservationQ, 20L);
     csConf.setMaximumCapacity(reservationQ, 40);
     csConf.setAverageCapacity(reservationQ, 20);
@@ -153,155 +141,51 @@ public class TestCapacitySchedulerPlanFollower {
     testPlanFollower(false);
   }
 
-  private void testPlanFollower(boolean isMove) throws PlanningException,
-      InterruptedException, AccessControlException {
-    // Initialize plan based on move flag
-    plan =
-        new InMemoryPlan(scheduler.getRootQueueMetrics(), policy, mAgent,
-            scheduler.getClusterResource(), 1L, res,
-            scheduler.getMinimumResourceCapability(), maxAlloc, "dedicated",
-            null, isMove);
-
-    // add a few reservations to the plan
-    long ts = System.currentTimeMillis();
-    ReservationId r1 = ReservationId.newInstance(ts, 1);
-    int[] f1 = { 10, 10, 10, 10, 10 };
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(r1, null, "u3",
-            "dedicated", 0, 0 + f1.length, ReservationSystemTestUtil
-                .generateAllocation(0L, 1L, f1), res, minAlloc)));
+  @Override
+  protected void verifyCapacity(Queue defQ) {
+    CSQueue csQueue = (CSQueue)defQ;
+    assertTrue(csQueue.getCapacity() > 0.9);
+  }
 
-    ReservationId r2 = ReservationId.newInstance(ts, 2);
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(r2, null, "u3",
-            "dedicated", 3, 3 + f1.length, ReservationSystemTestUtil
-                .generateAllocation(3L, 1L, f1), res, minAlloc)));
+  @Override
+  protected Queue getDefaultQueue() {
+    return cs.getQueue("dedicated" + PlanQueue.DEFAULT_QUEUE_SUFFIX);
+  }
 
-    ReservationId r3 = ReservationId.newInstance(ts, 3);
-    int[] f2 = { 0, 10, 20, 10, 0 };
-    assertTrue(plan.toString(),
-        plan.addReservation(new InMemoryReservationAllocation(r3, null, "u4",
-            "dedicated", 10, 10 + f2.length, ReservationSystemTestUtil
-                .generateAllocation(10L, 1L, f2), res, minAlloc)));
+  @Override
+  protected int getNumberOfApplications(Queue queue) {
+    CSQueue csQueue = (CSQueue)queue;
+    int numberOfApplications = csQueue.getNumApplications();
+    return numberOfApplications;
+  }
 
+  @Override
+  protected CapacitySchedulerPlanFollower createPlanFollower() {
     CapacitySchedulerPlanFollower planFollower =
         new CapacitySchedulerPlanFollower();
     planFollower.init(mClock, scheduler, Collections.singletonList(plan));
+    return planFollower;
+  }
 
-    when(mClock.getTime()).thenReturn(0L);
-    planFollower.run();
-
-    CSQueue defQ =
-        scheduler.getQueue("dedicated" + PlanQueue.DEFAULT_QUEUE_SUFFIX);
-    CSQueue q = scheduler.getQueue(r1.toString());
+  @Override
+  protected void assertReservationQueueExists(ReservationId r) {
+    CSQueue q = cs.getQueue(r.toString());
     assertNotNull(q);
-    // submit an app to r1
-    String user_0 = "test-user";
-    ApplicationId appId = ApplicationId.newInstance(0, 1);
-    ApplicationAttemptId appAttemptId_0 =
-        ApplicationAttemptId.newInstance(appId, 0);
-    AppAddedSchedulerEvent addAppEvent =
-        new AppAddedSchedulerEvent(appId, q.getQueueName(), user_0);
-    scheduler.handle(addAppEvent);
-    AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
-        new AppAttemptAddedSchedulerEvent(appAttemptId_0, false);
-    scheduler.handle(appAttemptAddedEvent);
-
-    // initial default reservation queue should have no apps
-    Assert.assertEquals(0, defQ.getNumApplications());
-
-    Assert.assertEquals(0.1, q.getCapacity(), 0.01);
-    Assert.assertEquals(0.1, q.getMaximumCapacity(), 1.0);
-    Assert.assertEquals(1, q.getNumApplications());
-
-    CSQueue q2 = scheduler.getQueue(r2.toString());
-    assertNull(q2);
-    CSQueue q3 = scheduler.getQueue(r3.toString());
-    assertNull(q3);
-
-    when(mClock.getTime()).thenReturn(3L);
-    planFollower.run();
+  }
 
-    Assert.assertEquals(0, defQ.getNumApplications());
-    q = scheduler.getQueue(r1.toString());
+  @Override
+  protected void assertReservationQueueExists(ReservationId r2,
+      double expectedCapacity, double expectedMaxCapacity) {
+    CSQueue q = cs.getQueue(r2.toString());
     assertNotNull(q);
-    Assert.assertEquals(0.1, q.getCapacity(), 0.01);
-    Assert.assertEquals(0.1, q.getMaximumCapacity(), 1.0);
-    Assert.assertEquals(1, q.getNumApplications());
-    q2 = scheduler.getQueue(r2.toString());
-    assertNotNull(q2);
-    Assert.assertEquals(0.1, q.getCapacity(), 0.01);
-    Assert.assertEquals(0.1, q.getMaximumCapacity(), 1.0);
-    q3 = scheduler.getQueue(r3.toString());
-    assertNull(q3);
-
-    when(mClock.getTime()).thenReturn(10L);
-    planFollower.run();
-
-    q = scheduler.getQueue(r1.toString());
-    if (isMove) {
-      // app should have been moved to default reservation queue
-      Assert.assertEquals(1, defQ.getNumApplications());
-      assertNull(q);
-    } else {
-      // app should be killed
-      Assert.assertEquals(0, defQ.getNumApplications());
-      assertNotNull(q);
-      AppAttemptRemovedSchedulerEvent appAttemptRemovedEvent =
-          new AppAttemptRemovedSchedulerEvent(appAttemptId_0,
-              RMAppAttemptState.KILLED, false);
-      scheduler.handle(appAttemptRemovedEvent);
-    }
-    q2 = scheduler.getQueue(r2.toString());
-    assertNull(q2);
-    q3 = scheduler.getQueue(r3.toString());
-    assertNotNull(q3);
-    Assert.assertEquals(0, q3.getCapacity(), 0.01);
-    Assert.assertEquals(1.0, q3.getMaximumCapacity(), 1.0);
-
-    when(mClock.getTime()).thenReturn(11L);
-    planFollower.run();
-
-    if (isMove) {
-      // app should have been moved to default reservation queue
-      Assert.assertEquals(1, defQ.getNumApplications());
-    } else {
-      // app should be killed
-      Assert.assertEquals(0, defQ.getNumApplications());
-    }
-    q = scheduler.getQueue(r1.toString());
-    assertNull(q);
-    q2 = scheduler.getQueue(r2.toString());
-    assertNull(q2);
-    q3 = scheduler.getQueue(r3.toString());
-    assertNotNull(q3);
-    Assert.assertEquals(0.1, q3.getCapacity(), 0.01);
-    Assert.assertEquals(0.1, q3.getMaximumCapacity(), 1.0);
-
-    when(mClock.getTime()).thenReturn(12L);
-    planFollower.run();
-
-    q = scheduler.getQueue(r1.toString());
-    assertNull(q);
-    q2 = scheduler.getQueue(r2.toString());
-    assertNull(q2);
-    q3 = scheduler.getQueue(r3.toString());
-    assertNotNull(q3);
-    Assert.assertEquals(0.2, q3.getCapacity(), 0.01);
-    Assert.assertEquals(0.2, q3.getMaximumCapacity(), 1.0);
-
-    when(mClock.getTime()).thenReturn(16L);
-    planFollower.run();
+    Assert.assertEquals(expectedCapacity, q.getCapacity(), 0.01);
+    Assert.assertEquals(expectedMaxCapacity, q.getMaximumCapacity(), 1.0);
+  }
 
-    q = scheduler.getQueue(r1.toString());
-    assertNull(q);
-    q2 = scheduler.getQueue(r2.toString());
+  @Override
+  protected void assertReservationQueueDoesNotExist(ReservationId r2) {
+    CSQueue q2 = cs.getQueue(r2.toString());
     assertNull(q2);
-    q3 = scheduler.getQueue(r3.toString());
-    assertNull(q3);
-
-    assertTrue(defQ.getCapacity() > 0.9);
-
   }
 
   public static ApplicationACLsManager mockAppACLsManager() {
@@ -312,8 +196,11 @@ public class TestCapacitySchedulerPlanFollower {
   @After
   public void tearDown() throws Exception {
     if (scheduler != null) {
-      scheduler.stop();
+      cs.stop();
     }
   }
 
+  protected Queue getReservationQueue(String reservationId) {
+    return cs.getQueue(reservationId);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e28fa62/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestSchedulerPlanFollowerBase.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestSchedulerPlanFollowerBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestSchedulerPlanFollowerBase.java
new file mode 100644
index 0000000..50df8fe
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestSchedulerPlanFollowerBase.java
@@ -0,0 +1,191 @@
+/*
+ * 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.reservation;
+
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.junit.Assert;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.when;
+
+public abstract class TestSchedulerPlanFollowerBase {
+  final static int GB = 1024;
+  protected Clock mClock = null;
+  protected ResourceScheduler scheduler = null;
+  protected ReservationAgent mAgent;
+  protected Resource minAlloc = Resource.newInstance(GB, 1);
+  protected Resource maxAlloc = Resource.newInstance(GB * 8, 8);
+  protected CapacityOverTimePolicy policy = new CapacityOverTimePolicy();
+  protected Plan plan;
+  private ResourceCalculator res = new DefaultResourceCalculator();
+
+  protected void testPlanFollower(boolean isMove) throws PlanningException,
+      InterruptedException, AccessControlException {
+    // Initialize plan based on move flag
+    plan =
+        new InMemoryPlan(scheduler.getRootQueueMetrics(), policy, mAgent,
+            scheduler.getClusterResource(), 1L, res,
+            scheduler.getMinimumResourceCapability(), maxAlloc, "dedicated",
+            null, isMove);
+
+    // add a few reservations to the plan
+    long ts = System.currentTimeMillis();
+    ReservationId r1 = ReservationId.newInstance(ts, 1);
+    int[] f1 = { 10, 10, 10, 10, 10 };
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(r1, null, "u3",
+            "dedicated", 0, 0 + f1.length, ReservationSystemTestUtil
+                .generateAllocation(0L, 1L, f1), res, minAlloc)));
+
+    ReservationId r2 = ReservationId.newInstance(ts, 2);
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(r2, null, "u3",
+            "dedicated", 3, 3 + f1.length, ReservationSystemTestUtil
+                .generateAllocation(3L, 1L, f1), res, minAlloc)));
+
+    ReservationId r3 = ReservationId.newInstance(ts, 3);
+    int[] f2 = { 0, 10, 20, 10, 0 };
+    assertTrue(plan.toString(),
+        plan.addReservation(new InMemoryReservationAllocation(r3, null, "u4",
+            "dedicated", 10, 10 + f2.length, ReservationSystemTestUtil
+                .generateAllocation(10L, 1L, f2), res, minAlloc)));
+
+    AbstractSchedulerPlanFollower planFollower = createPlanFollower();
+
+    when(mClock.getTime()).thenReturn(0L);
+    planFollower.run();
+
+    Queue q = getReservationQueue(r1.toString());
+    assertReservationQueueExists(r1);
+    // submit an app to r1
+    String user_0 = "test-user";
+    ApplicationId appId = ApplicationId.newInstance(0, 1);
+    ApplicationAttemptId appAttemptId_0 =
+        ApplicationAttemptId.newInstance(appId, 0);
+    AppAddedSchedulerEvent addAppEvent =
+        new AppAddedSchedulerEvent(appId, q.getQueueName(), user_0);
+    scheduler.handle(addAppEvent);
+    AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
+        new AppAttemptAddedSchedulerEvent(appAttemptId_0, false);
+    scheduler.handle(appAttemptAddedEvent);
+
+    // initial default reservation queue should have no apps
+
+    Queue defQ = getDefaultQueue();
+    Assert.assertEquals(0, getNumberOfApplications(defQ));
+
+    assertReservationQueueExists(r1, 0.1, 0.1);
+    Assert.assertEquals(1, getNumberOfApplications(q));
+
+    assertReservationQueueDoesNotExist(r2);
+    assertReservationQueueDoesNotExist(r3);
+
+    when(mClock.getTime()).thenReturn(3L);
+    planFollower.run();
+
+    Assert.assertEquals(0, getNumberOfApplications(defQ));
+    assertReservationQueueExists(r1, 0.1, 0.1);
+    Assert.assertEquals(1, getNumberOfApplications(q));
+    assertReservationQueueExists(r2, 0.1, 0.1);
+    assertReservationQueueDoesNotExist(r3);
+
+    when(mClock.getTime()).thenReturn(10L);
+    planFollower.run();
+
+    q = getReservationQueue(r1.toString());
+    if (isMove) {
+      // app should have been moved to default reservation queue
+      Assert.assertEquals(1, getNumberOfApplications(defQ));
+      assertNull(q);
+    } else {
+      // app should be killed
+      Assert.assertEquals(0, getNumberOfApplications(defQ));
+      assertNotNull(q);
+      AppAttemptRemovedSchedulerEvent appAttemptRemovedEvent =
+          new AppAttemptRemovedSchedulerEvent(appAttemptId_0,
+              RMAppAttemptState.KILLED, false);
+      scheduler.handle(appAttemptRemovedEvent);
+    }
+    assertReservationQueueDoesNotExist(r2);
+    assertReservationQueueExists(r3, 0, 1.0);
+
+    when(mClock.getTime()).thenReturn(11L);
+    planFollower.run();
+
+    if (isMove) {
+      // app should have been moved to default reservation queue
+      Assert.assertEquals(1, getNumberOfApplications(defQ));
+    } else {
+      // app should be killed
+      Assert.assertEquals(0, getNumberOfApplications(defQ));
+    }
+    assertReservationQueueDoesNotExist(r1);
+    assertReservationQueueDoesNotExist(r2);
+    assertReservationQueueExists(r3, 0.1, 0.1);
+
+    when(mClock.getTime()).thenReturn(12L);
+    planFollower.run();
+
+    assertReservationQueueDoesNotExist(r1);
+    assertReservationQueueDoesNotExist(r2);
+    assertReservationQueueExists(r3, 0.2, 0.2);
+
+    when(mClock.getTime()).thenReturn(16L);
+    planFollower.run();
+
+    assertReservationQueueDoesNotExist(r1);
+    assertReservationQueueDoesNotExist(r2);
+    assertReservationQueueDoesNotExist(r3);
+
+    verifyCapacity(defQ);
+  }
+
+  protected abstract Queue getReservationQueue(String reservationId);
+
+  protected abstract void verifyCapacity(Queue defQ);
+
+  protected abstract Queue getDefaultQueue();
+
+  protected abstract int getNumberOfApplications(Queue queue);
+
+  protected abstract AbstractSchedulerPlanFollower createPlanFollower();
+
+  protected abstract void assertReservationQueueExists(ReservationId r);
+
+  protected abstract void assertReservationQueueExists(ReservationId r2,
+      double expectedCapacity, double expectedMaxCapacity);
+
+  protected abstract void assertReservationQueueDoesNotExist(ReservationId r2);
+}


[12/18] hadoop git commit: YARN-2922. ConcurrentModificationException in CapacityScheduler's LeafQueue. Contributed by Rohith Sharmaks.

Posted by zh...@apache.org.
YARN-2922. ConcurrentModificationException in CapacityScheduler's LeafQueue. Contributed by Rohith Sharmaks.


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

Branch: refs/heads/HDFS-EC
Commit: 0c9446003fa9b462f75736d42c32925d931059c6
Parents: 10415a0
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Mon Jan 5 00:08:31 2015 +0900
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Jan 5 14:48:37 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../scheduler/capacity/LeafQueue.java           |  4 +-
 .../scheduler/capacity/TestLeafQueue.java       | 86 ++++++++++++++++++++
 3 files changed, 91 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c944600/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e6694f1..0d33b4a 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -308,6 +308,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2991. Fixed DrainDispatcher to reuse the draining code path in
     AsyncDispatcher. (Rohith Sharmaks via zjshen)
 
+    YARN-2922. ConcurrentModificationException in CapacityScheduler's LeafQueue.
+    (Rohith Sharmaks via ozawa)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c944600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
index f129ff4..47679a6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
@@ -1878,7 +1878,7 @@ public class LeafQueue extends AbstractCSQueue {
   }
 
   // return a single Resource capturing the overal amount of pending resources
-  public Resource getTotalResourcePending() {
+  public synchronized Resource getTotalResourcePending() {
     Resource ret = BuilderUtils.newResource(0, 0);
     for (FiCaSchedulerApp f : activeApplications) {
       Resources.addTo(ret, f.getTotalPendingRequests());
@@ -1887,7 +1887,7 @@ public class LeafQueue extends AbstractCSQueue {
   }
 
   @Override
-  public void collectSchedulerApplications(
+  public synchronized void collectSchedulerApplications(
       Collection<ApplicationAttemptId> apps) {
     for (FiCaSchedulerApp pendingApp : pendingApplications) {
       apps.add(pendingApp.getApplicationAttemptId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c944600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
index 642363e..fb7bb2c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
@@ -37,11 +37,13 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.ConcurrentModificationException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CyclicBarrier;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -61,6 +63,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
@@ -2353,6 +2356,89 @@ public class TestLeafQueue {
     }
   }
 
+  @Test
+  public void testConcurrentAccess() throws Exception {
+    YarnConfiguration conf = new YarnConfiguration();
+    MockRM rm = new MockRM();
+    rm.init(conf);
+    rm.start();
+
+    final String queue = "default";
+    final String user = "user";
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    final LeafQueue defaultQueue = (LeafQueue) cs.getQueue(queue);
+
+    final List<FiCaSchedulerApp> listOfApps =
+        createListOfApps(10000, user, defaultQueue);
+
+    final CyclicBarrier cb = new CyclicBarrier(2);
+    final List<ConcurrentModificationException> conException =
+        new ArrayList<ConcurrentModificationException>();
+
+    Thread submitAndRemove = new Thread(new Runnable() {
+
+      @Override
+      public void run() {
+
+        for (FiCaSchedulerApp fiCaSchedulerApp : listOfApps) {
+          defaultQueue.submitApplicationAttempt(fiCaSchedulerApp, user);
+        }
+        try {
+          cb.await();
+        } catch (Exception e) {
+          // Ignore
+        }
+        for (FiCaSchedulerApp fiCaSchedulerApp : listOfApps) {
+          defaultQueue.finishApplicationAttempt(fiCaSchedulerApp, queue);
+        }
+      }
+    }, "SubmitAndRemoveApplicationAttempt Thread");
+
+    Thread getAppsInQueue = new Thread(new Runnable() {
+      List<ApplicationAttemptId> apps = new ArrayList<ApplicationAttemptId>();
+
+      @Override
+      public void run() {
+        try {
+          try {
+            cb.await();
+          } catch (Exception e) {
+            // Ignore
+          }
+          defaultQueue.collectSchedulerApplications(apps);
+        } catch (ConcurrentModificationException e) {
+          conException.add(e);
+        }
+      }
+
+    }, "GetAppsInQueue Thread");
+
+    submitAndRemove.start();
+    getAppsInQueue.start();
+
+    submitAndRemove.join();
+    getAppsInQueue.join();
+
+    assertTrue("ConcurrentModificationException is thrown",
+        conException.isEmpty());
+    rm.stop();
+
+  }
+
+  private List<FiCaSchedulerApp> createListOfApps(int noOfApps, String user,
+      LeafQueue defaultQueue) {
+    List<FiCaSchedulerApp> appsLists = new ArrayList<FiCaSchedulerApp>();
+    for (int i = 0; i < noOfApps; i++) {
+      ApplicationAttemptId appAttemptId_0 =
+          TestUtils.getMockApplicationAttemptId(i, 0);
+      FiCaSchedulerApp app_0 =
+          new FiCaSchedulerApp(appAttemptId_0, user, defaultQueue,
+              mock(ActiveUsersManager.class), spyRMContext);
+      appsLists.add(app_0);
+    }
+    return appsLists;
+  }
+
   private CapacitySchedulerContext mockCSContext(
       CapacitySchedulerConfiguration csConf, Resource clusterResource) {
     CapacitySchedulerContext csContext = mock(CapacitySchedulerContext.class);