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 ji...@apache.org on 2015/05/14 22:12:02 UTC

[01/37] hadoop git commit: YARN-3489. RMServerUtils.validateResourceRequests should only obtain queue info once. (Varun Saxena via wangda)

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 444836b3d -> 15ccd967e


YARN-3489. RMServerUtils.validateResourceRequests should only obtain queue info once. (Varun Saxena via wangda)


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

Branch: refs/heads/HDFS-7240
Commit: d6f6741296639a73f5306e3ebefec84a40ca03e5
Parents: 444836b
Author: Wangda Tan <wa...@apache.org>
Authored: Mon May 11 17:31:15 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Mon May 11 17:31:15 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../server/resourcemanager/RMServerUtils.java   | 10 ++++++-
 .../scheduler/SchedulerUtils.java               | 30 +++++++++++++++-----
 3 files changed, 35 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6f67412/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 80cdf35..b5cb0a5 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -405,6 +405,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3243. CapacityScheduler should pass headroom from parent to children
     to make sure ParentQueue obey its capacity limits. (Wangda Tan via jianhe)
 
+    YARN-3489. RMServerUtils.validateResourceRequests should only obtain queue 
+    info once. (Varun Saxena via wangda)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6f67412/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.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/RMServerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
index 4669a28..4d2e41c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -94,9 +95,16 @@ public class RMServerUtils {
       Resource maximumResource, String queueName, YarnScheduler scheduler,
       RMContext rmContext)
       throws InvalidResourceRequestException {
+    // Get queue from scheduler
+    QueueInfo queueInfo = null;
+    try {
+      queueInfo = scheduler.getQueueInfo(queueName, false, false);
+    } catch (IOException e) {
+    }
+
     for (ResourceRequest resReq : ask) {
       SchedulerUtils.normalizeAndvalidateRequest(resReq, maximumResource,
-          queueName, scheduler, rmContext);
+          queueName, scheduler, rmContext, queueInfo);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6f67412/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.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/SchedulerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
index 0ef5c1e..8047d0b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
@@ -213,13 +213,21 @@ public class SchedulerUtils {
       Resource maximumResource, String queueName, YarnScheduler scheduler,
       boolean isRecovery, RMContext rmContext)
       throws InvalidResourceRequestException {
+    normalizeAndValidateRequest(resReq, maximumResource, queueName, scheduler,
+        isRecovery, rmContext, null);
+  }
 
-    QueueInfo queueInfo = null;
-    try {
-      queueInfo = scheduler.getQueueInfo(queueName, false, false);
-    } catch (IOException e) {
-      // it is possible queue cannot get when queue mapping is set, just ignore
-      // the queueInfo here, and move forward
+  public static void normalizeAndValidateRequest(ResourceRequest resReq,
+      Resource maximumResource, String queueName, YarnScheduler scheduler,
+      boolean isRecovery, RMContext rmContext, QueueInfo queueInfo)
+      throws InvalidResourceRequestException {
+    if (null == queueInfo) {
+      try {
+        queueInfo = scheduler.getQueueInfo(queueName, false, false);
+      } catch (IOException e) {
+        // it is possible queue cannot get when queue mapping is set, just ignore
+        // the queueInfo here, and move forward
+      }
     }
     SchedulerUtils.normalizeNodeLabelExpressionInRequest(resReq, queueInfo);
     if (!isRecovery) {
@@ -231,8 +239,16 @@ public class SchedulerUtils {
       Resource maximumResource, String queueName, YarnScheduler scheduler,
       RMContext rmContext)
       throws InvalidResourceRequestException {
+    normalizeAndvalidateRequest(resReq, maximumResource, queueName, scheduler,
+        rmContext, null);
+  }
+
+  public static void normalizeAndvalidateRequest(ResourceRequest resReq,
+      Resource maximumResource, String queueName, YarnScheduler scheduler,
+      RMContext rmContext, QueueInfo queueInfo)
+      throws InvalidResourceRequestException {
     normalizeAndValidateRequest(resReq, maximumResource, queueName, scheduler,
-        false, rmContext);
+        false, rmContext, queueInfo);
   }
 
   /**


[37/37] hadoop git commit: YARN-3505. Node's Log Aggregation Report with SUCCEED should not cached in RMApps. Contributed by Xuan Gong.

Posted by ji...@apache.org.
YARN-3505. Node's Log Aggregation Report with SUCCEED should not cached in RMApps. Contributed by Xuan Gong.


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

Branch: refs/heads/HDFS-7240
Commit: 15ccd967ee3e7046a50522089f67ba01f36ec76a
Parents: 05ff54c
Author: Junping Du <ju...@apache.org>
Authored: Thu May 14 10:57:36 2015 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Thu May 14 10:58:12 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../yarn/api/records/LogAggregationStatus.java  |   2 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |  10 +
 .../src/main/proto/yarn_protos.proto            |   1 +
 .../src/main/resources/yarn-default.xml         |   8 +
 .../protocolrecords/LogAggregationReport.java   |  16 +-
 .../protocolrecords/NodeHeartbeatRequest.java   |   7 +-
 .../impl/pb/LogAggregationReportPBImpl.java     |  40 ----
 .../impl/pb/NodeHeartbeatRequestPBImpl.java     |  82 +++----
 .../hadoop/yarn/server/webapp/AppBlock.java     |  19 +-
 .../yarn_server_common_service_protos.proto     |  14 +-
 .../nodemanager/NodeStatusUpdaterImpl.java      |  46 +---
 .../logaggregation/AppLogAggregatorImpl.java    |  19 +-
 .../server/resourcemanager/rmapp/RMAppImpl.java | 228 +++++++++++++++----
 .../resourcemanager/rmnode/RMNodeImpl.java      |  13 +-
 .../rmnode/RMNodeStatusEvent.java               |  11 +-
 .../resourcemanager/webapp/RMAppBlock.java      |  11 +-
 .../webapp/RMAppLogAggregationStatusBlock.java  |  37 ++-
 .../TestRMAppLogAggregationStatus.java          | 181 +++++++++++----
 19 files changed, 469 insertions(+), 279 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 0346c54..e0f2c52 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -111,6 +111,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3448. Added a rolling time-to-live LevelDB timeline store implementation.
     (Jonathan Eagles via zjshen)
 
+    YARN-3505. Node's Log Aggregation Report with SUCCEED should not cached in 
+    RMApps. (Xuan Gong via junping_du)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationStatus.java
index da1230c..1e10972 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LogAggregationStatus.java
@@ -34,6 +34,8 @@ public enum LogAggregationStatus {
   /** Log Aggregation is Running. */
   RUNNING,
 
+  /** Log Aggregation is Running, but has failures in previous cycles. */
+  RUNNING_WITH_FAILURE,
   /**
    * Log Aggregation is Succeeded. All of the logs have been aggregated
    * successfully.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 94f3e60..52fff14 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -718,6 +718,16 @@ public class YarnConfiguration extends Configuration {
       + "proxy-user-privileges.enabled";
   public static boolean DEFAULT_RM_PROXY_USER_PRIVILEGES_ENABLED = false;
 
+  /**
+   * How many diagnostics/failure messages can be saved in RM for
+   * log aggregation. It also defines the number of diagnostics/failure
+   * messages can be shown in log aggregation web ui.
+   */
+  public static final String RM_MAX_LOG_AGGREGATION_DIAGNOSTICS_IN_MEMORY =
+      RM_PREFIX + "max-log-aggregation-diagnostics-in-memory";
+  public static final int DEFAULT_RM_MAX_LOG_AGGREGATION_DIAGNOSTICS_IN_MEMORY =
+      10;
+
   /** Whether to enable log aggregation */
   public static final String LOG_AGGREGATION_ENABLED = YARN_PREFIX
       + "log-aggregation-enable";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index c45081a..4095676 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -204,6 +204,7 @@ enum LogAggregationStatusProto {
   LOG_SUCCEEDED = 4;
   LOG_FAILED = 5;
   LOG_TIME_OUT = 6;
+  LOG_RUNNING_WITH_FAILURE = 7;
 }
 
 message ApplicationAttemptReportProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 4d74f76..1dd88bd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -674,6 +674,14 @@
     <value>10</value>
   </property>
 
+  <property>
+    <description>Number of diagnostics/failure messages can be saved in RM for
+    log aggregation. It also defines the number of diagnostics/failure
+    messages can be shown in log aggregation web ui.</description>
+    <name>yarn.resourcemanager.max-log-aggregation-diagnostics-in-memory</name>
+    <value>10</value>
+  </property>
+
   <!-- Node Manager Configs -->
   <property>
     <description>The hostname of the NM.</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/LogAggregationReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/LogAggregationReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/LogAggregationReport.java
index b2270d8..d76f4cd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/LogAggregationReport.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/LogAggregationReport.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
-import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
@@ -32,7 +31,6 @@ import org.apache.hadoop.yarn.util.Records;
  * It includes details such as:
  * <ul>
  *   <li>{@link ApplicationId} of the application.</li>
- *   <li>{@link NodeId} of the NodeManager.</li>
  *   <li>{@link LogAggregationStatus}</li>
  *   <li>Diagnostic information</li>
  * </ul>
@@ -45,7 +43,7 @@ public abstract class LogAggregationReport {
   @Public
   @Unstable
   public static LogAggregationReport newInstance(ApplicationId appId,
-      NodeId nodeId, LogAggregationStatus status, String diagnosticMessage) {
+      LogAggregationStatus status, String diagnosticMessage) {
     LogAggregationReport report = Records.newRecord(LogAggregationReport.class);
     report.setApplicationId(appId);
     report.setLogAggregationStatus(status);
@@ -66,18 +64,6 @@ public abstract class LogAggregationReport {
   public abstract void setApplicationId(ApplicationId appId);
 
   /**
-   * Get the <code>NodeId</code>.
-   * @return <code>NodeId</code>
-   */
-  @Public
-  @Unstable
-  public abstract NodeId getNodeId();
-
-  @Public
-  @Unstable
-  public abstract void setNodeId(NodeId nodeId);
-
-  /**
    * Get the <code>LogAggregationStatus</code>.
    * @return <code>LogAggregationStatus</code>
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java
index 227363f..767e4b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatRequest.java
@@ -18,10 +18,9 @@
 
 package org.apache.hadoop.yarn.server.api.protocolrecords;
 
-import java.util.Map;
+import java.util.List;
 import java.util.Set;
 
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 import org.apache.hadoop.yarn.util.Records;
@@ -54,9 +53,9 @@ public abstract class NodeHeartbeatRequest {
   public abstract Set<String> getNodeLabels();
   public abstract void setNodeLabels(Set<String> nodeLabels);
 
-  public abstract Map<ApplicationId, LogAggregationReport>
+  public abstract List<LogAggregationReport>
       getLogAggregationReportsForApps();
 
   public abstract void setLogAggregationReportsForApps(
-      Map<ApplicationId, LogAggregationReport> logAggregationReportsForApps);
+      List<LogAggregationReport> logAggregationReportsForApps);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/LogAggregationReportPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/LogAggregationReportPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/LogAggregationReportPBImpl.java
index 75b6eab..ac6ad2e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/LogAggregationReportPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/LogAggregationReportPBImpl.java
@@ -22,13 +22,10 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
-import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LogAggregationStatusProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.LogAggregationReportProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.LogAggregationReportProtoOrBuilder;
 import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
@@ -45,7 +42,6 @@ public class LogAggregationReportPBImpl extends LogAggregationReport {
   boolean viaProto = false;
 
   private ApplicationId applicationId;
-  private NodeId nodeId;
 
   public LogAggregationReportPBImpl() {
     builder = LogAggregationReportProto.newBuilder();
@@ -89,12 +85,6 @@ public class LogAggregationReportPBImpl extends LogAggregationReport {
           builder.getApplicationId())) {
       builder.setApplicationId(convertToProtoFormat(this.applicationId));
     }
-
-    if (this.nodeId != null
-        && !((NodeIdPBImpl) this.nodeId).getProto().equals(
-          builder.getNodeId())) {
-      builder.setNodeId(convertToProtoFormat(this.nodeId));
-    }
   }
 
   private void mergeLocalToProto() {
@@ -191,34 +181,4 @@ public class LogAggregationReportPBImpl extends LogAggregationReport {
     }
     builder.setDiagnostics(diagnosticMessage);
   }
-
-  @Override
-  public NodeId getNodeId() {
-    if (this.nodeId != null) {
-      return this.nodeId;
-    }
-
-    LogAggregationReportProtoOrBuilder p = viaProto ? proto : builder;
-    if (!p.hasNodeId()) {
-      return null;
-    }
-    this.nodeId = convertFromProtoFormat(p.getNodeId());
-    return this.nodeId;
-  }
-
-  @Override
-  public void setNodeId(NodeId nodeId) {
-    maybeInitBuilder();
-    if (nodeId == null)
-      builder.clearNodeId();
-    this.nodeId = nodeId;
-  }
-
-  private NodeIdProto convertToProtoFormat(NodeId t) {
-    return ((NodeIdPBImpl) t).getProto();
-  }
-
-  private NodeIdPBImpl convertFromProtoFormat(NodeIdProto nodeId) {
-    return new NodeIdPBImpl(nodeId);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
index 03db39c..81f173d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
@@ -18,21 +18,16 @@
 
 package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
 
-import java.util.HashMap;
+import java.util.ArrayList;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Set;
 
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
-import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.StringArrayProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.LogAggregationReportProto;
-import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.LogAggregationReportsForAppsProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatRequestProtoOrBuilder;
 import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
@@ -51,9 +46,8 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
   private MasterKey lastKnownContainerTokenMasterKey = null;
   private MasterKey lastKnownNMTokenMasterKey = null;
   private Set<String> labels = null;
-  private Map<ApplicationId, LogAggregationReport>
-      logAggregationReportsForApps = null;
-  
+  private List<LogAggregationReport> logAggregationReportsForApps = null;
+
   public NodeHeartbeatRequestPBImpl() {
     builder = NodeHeartbeatRequestProto.newBuilder();
   }
@@ -110,12 +104,35 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
   private void addLogAggregationStatusForAppsToProto() {
     maybeInitBuilder();
     builder.clearLogAggregationReportsForApps();
-    for (Entry<ApplicationId, LogAggregationReport> entry : logAggregationReportsForApps
-      .entrySet()) {
-      builder.addLogAggregationReportsForApps(LogAggregationReportsForAppsProto
-        .newBuilder().setAppId(convertToProtoFormat(entry.getKey()))
-        .setLogAggregationReport(convertToProtoFormat(entry.getValue())));
+    if (this.logAggregationReportsForApps == null) {
+      return;
     }
+    Iterable<LogAggregationReportProto> it =
+        new Iterable<LogAggregationReportProto>() {
+          @Override
+          public Iterator<LogAggregationReportProto> iterator() {
+            return new Iterator<LogAggregationReportProto>() {
+              private Iterator<LogAggregationReport> iter =
+                  logAggregationReportsForApps.iterator();
+
+              @Override
+              public boolean hasNext() {
+                return iter.hasNext();
+              }
+
+              @Override
+              public LogAggregationReportProto next() {
+                return convertToProtoFormat(iter.next());
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+            };
+          }
+        };
+    builder.addAllLogAggregationReportsForApps(it);
   }
 
   private LogAggregationReportProto convertToProtoFormat(
@@ -246,17 +263,8 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
     labels = new HashSet<String>(nodeLabels.getElementsList());
   }
 
-  private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) {
-    return new ApplicationIdPBImpl(p);
-  }
-
-  private ApplicationIdProto convertToProtoFormat(ApplicationId t) {
-    return ((ApplicationIdPBImpl) t).getProto();
-  }
-
   @Override
-  public Map<ApplicationId, LogAggregationReport>
-      getLogAggregationReportsForApps() {
+  public List<LogAggregationReport> getLogAggregationReportsForApps() {
     if (this.logAggregationReportsForApps != null) {
       return this.logAggregationReportsForApps;
     }
@@ -266,15 +274,11 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
 
   private void initLogAggregationReportsForApps() {
     NodeHeartbeatRequestProtoOrBuilder p = viaProto ? proto : builder;
-    List<LogAggregationReportsForAppsProto> list =
+    List<LogAggregationReportProto> list =
         p.getLogAggregationReportsForAppsList();
-    this.logAggregationReportsForApps =
-        new HashMap<ApplicationId, LogAggregationReport>();
-    for (LogAggregationReportsForAppsProto c : list) {
-      ApplicationId appId = convertFromProtoFormat(c.getAppId());
-      LogAggregationReport report =
-          convertFromProtoFormat(c.getLogAggregationReport());
-      this.logAggregationReportsForApps.put(appId, report);
+    this.logAggregationReportsForApps = new ArrayList<LogAggregationReport>();
+    for (LogAggregationReportProto c : list) {
+      this.logAggregationReportsForApps.add(convertFromProtoFormat(c));
     }
   }
 
@@ -285,14 +289,10 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
 
   @Override
   public void setLogAggregationReportsForApps(
-      Map<ApplicationId, LogAggregationReport> logAggregationStatusForApps) {
-    if (logAggregationStatusForApps == null
-        || logAggregationStatusForApps.isEmpty()) {
-      return;
+      List<LogAggregationReport> logAggregationStatusForApps) {
+    if(logAggregationStatusForApps == null) {
+      builder.clearLogAggregationReportsForApps();
     }
-    maybeInitBuilder();
-    this.logAggregationReportsForApps =
-        new HashMap<ApplicationId, LogAggregationReport>();
-    this.logAggregationReportsForApps.putAll(logAggregationStatusForApps);
+    this.logAggregationReportsForApps = logAggregationStatusForApps;
   }
 }  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
index dd5a4c8..f46197e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.ContainerNotFoundException;
@@ -192,8 +193,17 @@ public class AppBlock extends HtmlBlock {
             : "ApplicationMaster");
     if (webUiType != null
         && webUiType.equals(YarnWebParams.RM_WEB_UI)) {
-      overviewTable._("Log Aggregation Status",
-        root_url("logaggregationstatus", app.getAppId()), "Status");
+      LogAggregationStatus status = getLogAggregationStatus();
+      if (status == null) {
+        overviewTable._("Log Aggregation Status", "N/A");
+      } else if (status == LogAggregationStatus.DISABLED
+          || status == LogAggregationStatus.NOT_START
+          || status == LogAggregationStatus.SUCCEEDED) {
+        overviewTable._("Log Aggregation Status", status.name());
+      } else {
+        overviewTable._("Log Aggregation Status",
+            root_url("logaggregationstatus", app.getAppId()), status.name());
+      }
     }
     overviewTable._("Diagnostics:",
         app.getDiagnosticsInfo() == null ? "" : app.getDiagnosticsInfo());
@@ -342,4 +352,9 @@ public class AppBlock extends HtmlBlock {
   protected void createApplicationMetricsTable(Block html) {
 
   }
+
+  // This will be overrided in RMAppBlock
+  protected LogAggregationStatus getLogAggregationStatus() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
index d34c9f7..c027ac0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
@@ -50,19 +50,13 @@ message NodeHeartbeatRequestProto {
   optional MasterKeyProto last_known_container_token_master_key = 2;
   optional MasterKeyProto last_known_nm_token_master_key = 3;
   optional StringArrayProto nodeLabels = 4;
-  repeated LogAggregationReportsForAppsProto log_aggregation_reports_for_apps = 5;
-}
-
-message LogAggregationReportsForAppsProto {
-  optional ApplicationIdProto appId = 1;
-  optional LogAggregationReportProto log_aggregation_report = 2;
+  repeated LogAggregationReportProto log_aggregation_reports_for_apps = 5;
 }
 
 message LogAggregationReportProto {
-optional ApplicationIdProto application_id = 1;
-optional NodeIdProto node_id = 2;
-optional LogAggregationStatusProto log_aggregation_status = 3;
-optional string diagnostics = 4 [default = "N/A"];
+  optional ApplicationIdProto application_id = 1;
+  optional LogAggregationStatusProto log_aggregation_status = 2;
+  optional string diagnostics = 3 [default = "N/A"];
 }
 
 message NodeHeartbeatResponseProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index 0eb7ff4..8046228 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -75,7 +75,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeLabelsProvider;
-import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -666,7 +665,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
 
             if (logAggregationEnabled) {
               // pull log aggregation status for application running in this NM
-              Map<ApplicationId, LogAggregationReport> logAggregationReports =
+              List<LogAggregationReport> logAggregationReports =
                   getLogAggregationReportsForApps(context
                     .getLogAggregationStatusForApps());
               if (logAggregationReports != null
@@ -810,47 +809,14 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     statusUpdater.start();
   }
 
-  private Map<ApplicationId, LogAggregationReport>
-      getLogAggregationReportsForApps(
-          ConcurrentLinkedQueue<LogAggregationReport> lastestLogAggregationStatus) {
-    Map<ApplicationId, LogAggregationReport> latestLogAggregationReports =
-        new HashMap<ApplicationId, LogAggregationReport>();
+  private List<LogAggregationReport> getLogAggregationReportsForApps(
+      ConcurrentLinkedQueue<LogAggregationReport> lastestLogAggregationStatus) {
     LogAggregationReport status;
     while ((status = lastestLogAggregationStatus.poll()) != null) {
       this.logAggregationReportForAppsTempList.add(status);
     }
-    for (LogAggregationReport logAggregationReport
-        : this.logAggregationReportForAppsTempList) {
-      LogAggregationReport report = null;
-      if (latestLogAggregationReports.containsKey(logAggregationReport
-        .getApplicationId())) {
-        report =
-            latestLogAggregationReports.get(logAggregationReport
-              .getApplicationId());
-        report.setLogAggregationStatus(logAggregationReport
-          .getLogAggregationStatus());
-        String message = report.getDiagnosticMessage();
-        if (logAggregationReport.getDiagnosticMessage() != null
-            && !logAggregationReport.getDiagnosticMessage().isEmpty()) {
-          if (message != null) {
-            message += logAggregationReport.getDiagnosticMessage();
-          } else {
-            message = logAggregationReport.getDiagnosticMessage();
-          }
-          report.setDiagnosticMessage(message);
-        }
-      } else {
-        report = Records.newRecord(LogAggregationReport.class);
-        report.setApplicationId(logAggregationReport.getApplicationId());
-        report.setNodeId(this.nodeId);
-        report.setLogAggregationStatus(logAggregationReport
-          .getLogAggregationStatus());
-        report
-          .setDiagnosticMessage(logAggregationReport.getDiagnosticMessage());
-      }
-      latestLogAggregationReports.put(logAggregationReport.getApplicationId(),
-        report);
-    }
-    return latestLogAggregationReports;
+    List<LogAggregationReport> reports = new ArrayList<LogAggregationReport>();
+    reports.addAll(logAggregationReportForAppsTempList);
+    return reports;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
index 3111f10..dd2ab25 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
@@ -306,6 +306,7 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
                     + currentTime);
 
       String diagnosticMessage = "";
+      boolean logAggregationSucceedInThisCycle = true;
       final boolean rename = uploadedLogsInThisCycle;
       try {
         userUgi.doAs(new PrivilegedExceptionAction<Object>() {
@@ -338,20 +339,28 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
                 + LogAggregationUtils.getNodeString(nodeId) + " at "
                 + Times.format(currentTime) + "\n";
         renameTemporaryLogFileFailed = true;
+        logAggregationSucceedInThisCycle = false;
       }
 
       LogAggregationReport report =
           Records.newRecord(LogAggregationReport.class);
       report.setApplicationId(appId);
-      report.setNodeId(nodeId);
       report.setDiagnosticMessage(diagnosticMessage);
+      report.setLogAggregationStatus(logAggregationSucceedInThisCycle
+          ? LogAggregationStatus.RUNNING
+          : LogAggregationStatus.RUNNING_WITH_FAILURE);
+      this.context.getLogAggregationStatusForApps().add(report);
       if (appFinished) {
-        report.setLogAggregationStatus(renameTemporaryLogFileFailed
+        // If the app is finished, one extra final report with log aggregation
+        // status SUCCEEDED/FAILED will be sent to RM to inform the RM
+        // that the log aggregation in this NM is completed.
+        LogAggregationReport finalReport =
+            Records.newRecord(LogAggregationReport.class);
+        finalReport.setApplicationId(appId);
+        finalReport.setLogAggregationStatus(renameTemporaryLogFileFailed
             ? LogAggregationStatus.FAILED : LogAggregationStatus.SUCCEEDED);
-      } else {
-        report.setLogAggregationStatus(LogAggregationStatus.RUNNING);
+        this.context.getLogAggregationStatusForApps().add(report);
       }
-      this.context.getLogAggregationStatusForApps().add(report);
     } finally {
       if (writer != null) {
         writer.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 8abc478..f3dacd6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -22,12 +22,15 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -36,6 +39,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -152,6 +156,13 @@ public class RMAppImpl implements RMApp, Recoverable {
   private final Map<NodeId, LogAggregationReport> logAggregationStatus =
       new HashMap<NodeId, LogAggregationReport>();
   private LogAggregationStatus logAggregationStatusForAppReport;
+  private int logAggregationSucceed = 0;
+  private int logAggregationFailed = 0;
+  private Map<NodeId, List<String>> logAggregationDiagnosticsForNMs =
+      new HashMap<NodeId, List<String>>();
+  private Map<NodeId, List<String>> logAggregationFailureMessagesForNMs =
+      new HashMap<NodeId, List<String>>();
+  private final int maxLogAggregationDiagnosticsInMemory;
 
   // These states stored are only valid when app is at killing or final_saving.
   private RMAppState stateBeforeKilling;
@@ -437,6 +448,14 @@ public class RMAppImpl implements RMApp, Recoverable {
     this.logAggregationEnabled =
         conf.getBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED,
           YarnConfiguration.DEFAULT_LOG_AGGREGATION_ENABLED);
+    if (this.logAggregationEnabled) {
+      this.logAggregationStatusForAppReport = LogAggregationStatus.NOT_START;
+    } else {
+      this.logAggregationStatusForAppReport = LogAggregationStatus.DISABLED;
+    }
+    maxLogAggregationDiagnosticsInMemory = conf.getInt(
+        YarnConfiguration.RM_MAX_LOG_AGGREGATION_DIAGNOSTICS_IN_MEMORY,
+        YarnConfiguration.DEFAULT_RM_MAX_LOG_AGGREGATION_DIAGNOSTICS_IN_MEMORY);
   }
 
   @Override
@@ -834,10 +853,9 @@ public class RMAppImpl implements RMApp, Recoverable {
 
       if (!app.logAggregationStatus.containsKey(nodeAddedEvent.getNodeId())) {
         app.logAggregationStatus.put(nodeAddedEvent.getNodeId(),
-          LogAggregationReport.newInstance(app.applicationId, nodeAddedEvent
-            .getNodeId(), app.logAggregationEnabled
-              ? LogAggregationStatus.NOT_START : LogAggregationStatus.DISABLED,
-            ""));
+          LogAggregationReport.newInstance(app.applicationId,
+            app.logAggregationEnabled ? LogAggregationStatus.NOT_START
+                : LogAggregationStatus.DISABLED, ""));
       }
     };
   }
@@ -1401,18 +1419,20 @@ public class RMAppImpl implements RMApp, Recoverable {
       Map<NodeId, LogAggregationReport> outputs =
           new HashMap<NodeId, LogAggregationReport>();
       outputs.putAll(logAggregationStatus);
-      for (Entry<NodeId, LogAggregationReport> output : outputs.entrySet()) {
-        if (!output.getValue().getLogAggregationStatus()
-          .equals(LogAggregationStatus.TIME_OUT)
-            && !output.getValue().getLogAggregationStatus()
-              .equals(LogAggregationStatus.SUCCEEDED)
-            && !output.getValue().getLogAggregationStatus()
-              .equals(LogAggregationStatus.FAILED)
-            && isAppInFinalState(this)
-            && System.currentTimeMillis() > this.logAggregationStartTime
-                + this.logAggregationStatusTimeout) {
-          output.getValue().setLogAggregationStatus(
-            LogAggregationStatus.TIME_OUT);
+      if (!isLogAggregationFinished()) {
+        for (Entry<NodeId, LogAggregationReport> output : outputs.entrySet()) {
+          if (!output.getValue().getLogAggregationStatus()
+            .equals(LogAggregationStatus.TIME_OUT)
+              && !output.getValue().getLogAggregationStatus()
+                .equals(LogAggregationStatus.SUCCEEDED)
+              && !output.getValue().getLogAggregationStatus()
+                .equals(LogAggregationStatus.FAILED)
+              && isAppInFinalState(this)
+              && System.currentTimeMillis() > this.logAggregationStartTime
+                  + this.logAggregationStatusTimeout) {
+            output.getValue().setLogAggregationStatus(
+              LogAggregationStatus.TIME_OUT);
+          }
         }
       }
       return outputs;
@@ -1424,32 +1444,46 @@ public class RMAppImpl implements RMApp, Recoverable {
   public void aggregateLogReport(NodeId nodeId, LogAggregationReport report) {
     try {
       this.writeLock.lock();
-      if (this.logAggregationEnabled) {
+      if (this.logAggregationEnabled && !isLogAggregationFinished()) {
         LogAggregationReport curReport = this.logAggregationStatus.get(nodeId);
+        boolean stateChangedToFinal = false;
         if (curReport == null) {
           this.logAggregationStatus.put(nodeId, report);
+          if (isLogAggregationFinishedForNM(report)) {
+            stateChangedToFinal = true;
+          }
         } else {
-          if (curReport.getLogAggregationStatus().equals(
-            LogAggregationStatus.TIME_OUT)) {
-            if (report.getLogAggregationStatus().equals(
-              LogAggregationStatus.SUCCEEDED)
-                || report.getLogAggregationStatus().equals(
-                  LogAggregationStatus.FAILED)) {
-              curReport.setLogAggregationStatus(report
-                .getLogAggregationStatus());
+          if (isLogAggregationFinishedForNM(report)) {
+            if (!isLogAggregationFinishedForNM(curReport)) {
+              stateChangedToFinal = true;
             }
-          } else {
-            curReport.setLogAggregationStatus(report.getLogAggregationStatus());
           }
-
-          if (report.getDiagnosticMessage() != null
-              && !report.getDiagnosticMessage().isEmpty()) {
-            curReport
-              .setDiagnosticMessage(curReport.getDiagnosticMessage() == null
-                  ? report.getDiagnosticMessage() : curReport
-                    .getDiagnosticMessage() + report.getDiagnosticMessage());
+          if (report.getLogAggregationStatus() != LogAggregationStatus.RUNNING
+              || curReport.getLogAggregationStatus() !=
+                  LogAggregationStatus.RUNNING_WITH_FAILURE) {
+            if (curReport.getLogAggregationStatus()
+                == LogAggregationStatus.TIME_OUT
+                && report.getLogAggregationStatus()
+                    == LogAggregationStatus.RUNNING) {
+            // If the log aggregation status got from latest nm heartbeat
+            // is Running, and current log aggregation status is TimeOut,
+            // based on whether there are any failure messages for this NM,
+            // we will reset the log aggregation status as RUNNING or
+            // RUNNING_WITH_FAILURE
+              if (logAggregationFailureMessagesForNMs.get(nodeId) != null &&
+                  !logAggregationFailureMessagesForNMs.get(nodeId).isEmpty()) {
+                report.setLogAggregationStatus(
+                    LogAggregationStatus.RUNNING_WITH_FAILURE);
+              }
+            }
+            curReport.setLogAggregationStatus(report
+              .getLogAggregationStatus());
           }
         }
+        updateLogAggregationDiagnosticMessages(nodeId, report);
+        if (isAppInFinalState(this) && stateChangedToFinal) {
+          updateLogAggregationStatus(nodeId);
+        }
       }
     } finally {
       this.writeLock.unlock();
@@ -1458,29 +1492,32 @@ public class RMAppImpl implements RMApp, Recoverable {
 
   @Override
   public LogAggregationStatus getLogAggregationStatusForAppReport() {
-    if (!logAggregationEnabled) {
-      return LogAggregationStatus.DISABLED;
-    }
-    if (this.logAggregationStatusForAppReport == LogAggregationStatus.FAILED
-        || this.logAggregationStatusForAppReport == LogAggregationStatus.SUCCEEDED) {
-      return this.logAggregationStatusForAppReport;
-    }
     try {
       this.readLock.lock();
+      if (! logAggregationEnabled) {
+        return LogAggregationStatus.DISABLED;
+      }
+      if (isLogAggregationFinished()) {
+        return this.logAggregationStatusForAppReport;
+      }
       Map<NodeId, LogAggregationReport> reports =
           getLogAggregationReportsForApp();
       if (reports.size() == 0) {
-        return null;
+        return this.logAggregationStatusForAppReport;
       }
       int logNotStartCount = 0;
       int logCompletedCount = 0;
       int logTimeOutCount = 0;
       int logFailedCount = 0;
+      int logRunningWithFailure = 0;
       for (Entry<NodeId, LogAggregationReport> report : reports.entrySet()) {
         switch (report.getValue().getLogAggregationStatus()) {
           case NOT_START:
             logNotStartCount++;
             break;
+          case RUNNING_WITH_FAILURE:
+            logRunningWithFailure ++;
+            break;
           case SUCCEEDED:
             logCompletedCount++;
             break;
@@ -1506,19 +1543,122 @@ public class RMAppImpl implements RMApp, Recoverable {
         // the log aggregation is finished. And the log aggregation status will
         // not be updated anymore.
         if (logFailedCount > 0 && isAppInFinalState(this)) {
-          this.logAggregationStatusForAppReport = LogAggregationStatus.FAILED;
           return LogAggregationStatus.FAILED;
         } else if (logTimeOutCount > 0) {
           return LogAggregationStatus.TIME_OUT;
         }
         if (isAppInFinalState(this)) {
-          this.logAggregationStatusForAppReport = LogAggregationStatus.SUCCEEDED;
           return LogAggregationStatus.SUCCEEDED;
         }
+      } else if (logRunningWithFailure > 0) {
+        return LogAggregationStatus.RUNNING_WITH_FAILURE;
       }
       return LogAggregationStatus.RUNNING;
     } finally {
       this.readLock.unlock();
     }
   }
+
+  private boolean isLogAggregationFinished() {
+    return this.logAggregationStatusForAppReport
+      .equals(LogAggregationStatus.SUCCEEDED)
+        || this.logAggregationStatusForAppReport
+          .equals(LogAggregationStatus.FAILED);
+
+  }
+
+  private boolean isLogAggregationFinishedForNM(LogAggregationReport report) {
+    return report.getLogAggregationStatus() == LogAggregationStatus.SUCCEEDED
+        || report.getLogAggregationStatus() == LogAggregationStatus.FAILED;
+  }
+
+  private void updateLogAggregationDiagnosticMessages(NodeId nodeId,
+      LogAggregationReport report) {
+    if (report.getDiagnosticMessage() != null
+        && !report.getDiagnosticMessage().isEmpty()) {
+      if (report.getLogAggregationStatus()
+          == LogAggregationStatus.RUNNING ) {
+        List<String> diagnostics = logAggregationDiagnosticsForNMs.get(nodeId);
+        if (diagnostics == null) {
+          diagnostics = new ArrayList<String>();
+          logAggregationDiagnosticsForNMs.put(nodeId, diagnostics);
+        } else {
+          if (diagnostics.size()
+              == maxLogAggregationDiagnosticsInMemory) {
+            diagnostics.remove(0);
+          }
+        }
+        diagnostics.add(report.getDiagnosticMessage());
+        this.logAggregationStatus.get(nodeId).setDiagnosticMessage(
+          StringUtils.join(diagnostics, "\n"));
+      } else if (report.getLogAggregationStatus()
+          == LogAggregationStatus.RUNNING_WITH_FAILURE) {
+        List<String> failureMessages =
+            logAggregationFailureMessagesForNMs.get(nodeId);
+        if (failureMessages == null) {
+          failureMessages = new ArrayList<String>();
+          logAggregationFailureMessagesForNMs.put(nodeId, failureMessages);
+        } else {
+          if (failureMessages.size()
+              == maxLogAggregationDiagnosticsInMemory) {
+            failureMessages.remove(0);
+          }
+        }
+        failureMessages.add(report.getDiagnosticMessage());
+      }
+    }
+  }
+
+  private void updateLogAggregationStatus(NodeId nodeId) {
+    LogAggregationStatus status =
+        this.logAggregationStatus.get(nodeId).getLogAggregationStatus();
+    if (status.equals(LogAggregationStatus.SUCCEEDED)) {
+      this.logAggregationSucceed++;
+    } else if (status.equals(LogAggregationStatus.FAILED)) {
+      this.logAggregationFailed++;
+    }
+    if (this.logAggregationSucceed == this.logAggregationStatus.size()) {
+      this.logAggregationStatusForAppReport =
+          LogAggregationStatus.SUCCEEDED;
+      // Since the log aggregation status for this application for all NMs
+      // is SUCCEEDED, it means all logs are aggregated successfully.
+      // We could remove all the cached log aggregation reports
+      this.logAggregationStatus.clear();
+      this.logAggregationDiagnosticsForNMs.clear();
+      this.logAggregationFailureMessagesForNMs.clear();
+    } else if (this.logAggregationSucceed + this.logAggregationFailed
+        == this.logAggregationStatus.size()) {
+      this.logAggregationStatusForAppReport = LogAggregationStatus.FAILED;
+      // We have collected the log aggregation status for all NMs.
+      // The log aggregation status is FAILED which means the log
+      // aggregation fails in some NMs. We are only interested in the
+      // nodes where the log aggregation is failed. So we could remove
+      // the log aggregation details for those succeeded NMs
+      for (Iterator<Map.Entry<NodeId, LogAggregationReport>> it =
+          this.logAggregationStatus.entrySet().iterator(); it.hasNext();) {
+        Map.Entry<NodeId, LogAggregationReport> entry = it.next();
+        if (entry.getValue().getLogAggregationStatus()
+          .equals(LogAggregationStatus.SUCCEEDED)) {
+          it.remove();
+        }
+      }
+      // the log aggregation has finished/failed.
+      // and the status will not be updated anymore.
+      this.logAggregationDiagnosticsForNMs.clear();
+    }
+  }
+
+  public String getLogAggregationFailureMessagesForNM(NodeId nodeId) {
+    try {
+      this.readLock.lock();
+      List<String> failureMessages =
+          this.logAggregationFailureMessagesForNMs.get(nodeId);
+      if (failureMessages == null || failureMessages.isEmpty()) {
+        return StringUtils.EMPTY;
+      }
+      return StringUtils.join(failureMessages, "\n");
+    } finally {
+      this.readLock.unlock();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
index 3be1867..a11aacf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
@@ -22,8 +22,6 @@ import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.ConcurrentLinkedQueue;
@@ -777,7 +775,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
 
       rmNode.handleContainerStatus(statusEvent.getContainers());
 
-      Map<ApplicationId, LogAggregationReport> logAggregationReportsForApps =
+      List<LogAggregationReport> logAggregationReportsForApps =
           statusEvent.getLogAggregationReportsForApps();
       if (logAggregationReportsForApps != null
           && !logAggregationReportsForApps.isEmpty()) {
@@ -915,12 +913,11 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
   }
 
   private void handleLogAggregationStatus(
-      Map<ApplicationId, LogAggregationReport> logAggregationReportsForApps) {
-    for (Entry<ApplicationId, LogAggregationReport> report :
-        logAggregationReportsForApps.entrySet()) {
-      RMApp rmApp = this.context.getRMApps().get(report.getKey());
+      List<LogAggregationReport> logAggregationReportsForApps) {
+    for (LogAggregationReport report : logAggregationReportsForApps) {
+      RMApp rmApp = this.context.getRMApps().get(report.getApplicationId());
       if (rmApp != null) {
-        ((RMAppImpl)rmApp).aggregateLogReport(this.nodeId, report.getValue());
+        ((RMAppImpl)rmApp).aggregateLogReport(this.nodeId, report);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java
index 4bbf610..b95d7d3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java
@@ -19,8 +19,6 @@
 package org.apache.hadoop.yarn.server.resourcemanager.rmnode;
 
 import java.util.List;
-import java.util.Map;
-
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -34,7 +32,7 @@ public class RMNodeStatusEvent extends RMNodeEvent {
   private final List<ContainerStatus> containersCollection;
   private final NodeHeartbeatResponse latestResponse;
   private final List<ApplicationId> keepAliveAppIds;
-  private Map<ApplicationId, LogAggregationReport> logAggregationReportsForApps;
+  private List<LogAggregationReport> logAggregationReportsForApps;
 
   public RMNodeStatusEvent(NodeId nodeId, NodeHealthStatus nodeHealthStatus,
       List<ContainerStatus> collection, List<ApplicationId> keepAliveAppIds,
@@ -50,7 +48,7 @@ public class RMNodeStatusEvent extends RMNodeEvent {
   public RMNodeStatusEvent(NodeId nodeId, NodeHealthStatus nodeHealthStatus,
       List<ContainerStatus> collection, List<ApplicationId> keepAliveAppIds,
       NodeHeartbeatResponse latestResponse,
-      Map<ApplicationId, LogAggregationReport> logAggregationReportsForApps) {
+      List<LogAggregationReport> logAggregationReportsForApps) {
     super(nodeId, RMNodeEventType.STATUS_UPDATE);
     this.nodeHealthStatus = nodeHealthStatus;
     this.containersCollection = collection;
@@ -75,13 +73,12 @@ public class RMNodeStatusEvent extends RMNodeEvent {
     return this.keepAliveAppIds;
   }
 
-  public Map<ApplicationId, LogAggregationReport>
-      getLogAggregationReportsForApps() {
+  public List<LogAggregationReport> getLogAggregationReportsForApps() {
     return this.logAggregationReportsForApps;
   }
 
   public void setLogAggregationReportsForApps(
-      Map<ApplicationId, LogAggregationReport> logAggregationReportsForApps) {
+      List<LogAggregationReport> logAggregationReportsForApps) {
     this.logAggregationReportsForApps = logAggregationReportsForApps;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
index 43e26be..38e0e3b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
@@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
+import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -34,7 +35,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptInfo;
 import org.apache.hadoop.yarn.server.webapp.AppBlock;
-import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
@@ -170,4 +170,13 @@ public class RMAppBlock extends AppBlock{
 
     tbody._()._();
   }
+
+  @Override
+  protected LogAggregationStatus getLogAggregationStatus() {
+    RMApp rmApp = this.rm.getRMContext().getRMApps().get(appID);
+    if (rmApp == null) {
+      return null;
+    }
+    return rmApp.getLogAggregationStatusForAppReport();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppLogAggregationStatusBlock.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/RMAppLogAggregationStatusBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppLogAggregationStatusBlock.java
index a2f61e3..f7f7c97 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppLogAggregationStatusBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppLogAggregationStatusBlock.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
@@ -93,6 +94,9 @@ public class RMAppLogAggregationStatusBlock extends HtmlBlock {
       .td("Log Aggregation does not Start.")._();
     table_description.tr().td(LogAggregationStatus.RUNNING.name())
       .td("Log Aggregation is Running.")._();
+    table_description.tr().td(LogAggregationStatus.RUNNING_WITH_FAILURE.name())
+      .td("Log Aggregation is Running, but has failures "
+          + "in previous cycles")._();
     table_description.tr().td(LogAggregationStatus.SUCCEEDED.name())
       .td("Log Aggregation is Succeeded. All of the logs have been "
           + "aggregated successfully.")._();
@@ -106,24 +110,29 @@ public class RMAppLogAggregationStatusBlock extends HtmlBlock {
     table_description._();
     div_description._();
 
-    boolean logAggregationEnabled =
-        conf.getBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED,
-          YarnConfiguration.DEFAULT_LOG_AGGREGATION_ENABLED);
+    RMApp rmApp = rm.getRMContext().getRMApps().get(appId);
     // Application Log aggregation status Table
     DIV<Hamlet> div = html.div(_INFO_WRAP);
     TABLE<DIV<Hamlet>> table =
         div.h3(
           "Log Aggregation: "
-              + (logAggregationEnabled ? "Enabled" : "Disabled")).table(
+              + (rmApp == null ? "N/A" : rmApp
+                .getLogAggregationStatusForAppReport() == null ? "N/A" : rmApp
+                .getLogAggregationStatusForAppReport().name())).table(
           "#LogAggregationStatus");
-    table.
-      tr().
-        th(_TH, "NodeId").
-        th(_TH, "Log Aggregation Status").
-        th(_TH, "Diagnostis Message").
-      _();
 
-    RMApp rmApp = rm.getRMContext().getRMApps().get(appId);
+    int maxLogAggregationDiagnosticsInMemory = conf.getInt(
+      YarnConfiguration.RM_MAX_LOG_AGGREGATION_DIAGNOSTICS_IN_MEMORY,
+      YarnConfiguration.DEFAULT_RM_MAX_LOG_AGGREGATION_DIAGNOSTICS_IN_MEMORY);
+    table
+      .tr()
+      .th(_TH, "NodeId")
+      .th(_TH, "Log Aggregation Status")
+      .th(_TH, "Last "
+          + maxLogAggregationDiagnosticsInMemory + " Diagnostic Messages")
+      .th(_TH, "Last "
+          + maxLogAggregationDiagnosticsInMemory + " Failure Messages")._();
+
     if (rmApp != null) {
       Map<NodeId, LogAggregationReport> logAggregationReports =
           rmApp.getLogAggregationReportsForApp();
@@ -136,10 +145,14 @@ public class RMAppLogAggregationStatusBlock extends HtmlBlock {
           String message =
               report.getValue() == null ? null : report.getValue()
                 .getDiagnosticMessage();
+          String failureMessage =
+              report.getValue() == null ? null : ((RMAppImpl)rmApp)
+                  .getLogAggregationFailureMessagesForNM(report.getKey());
           table.tr()
             .td(report.getKey().toString())
             .td(status == null ? "N/A" : status.toString())
-            .td(message == null ? "N/A" : message)._();
+            .td(message == null ? "N/A" : message)
+            .td(failureMessage == null ? "N/A" : failureMessage)._();
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15ccd967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.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/logaggregationstatus/TestRMAppLogAggregationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java
index 4eec63f..9af4290 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java
@@ -23,7 +23,7 @@ import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.mock;
 
 import java.util.ArrayList;
-import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
@@ -155,26 +155,26 @@ public class TestRMAppLogAggregationStatus {
         .getLogAggregationStatus());
     }
 
-    Map<ApplicationId, LogAggregationReport> node1ReportForApp =
-        new HashMap<ApplicationId, LogAggregationReport>();
+    List<LogAggregationReport> node1ReportForApp =
+        new ArrayList<LogAggregationReport>();
     String messageForNode1_1 =
         "node1 logAggregation status updated at " + System.currentTimeMillis();
     LogAggregationReport report1 =
-        LogAggregationReport.newInstance(appId, nodeId1,
-          LogAggregationStatus.RUNNING, messageForNode1_1);
-    node1ReportForApp.put(appId, report1);
+        LogAggregationReport.newInstance(appId, LogAggregationStatus.RUNNING,
+          messageForNode1_1);
+    node1ReportForApp.add(report1);
     node1.handle(new RMNodeStatusEvent(node1.getNodeID(), NodeHealthStatus
       .newInstance(true, null, 0), new ArrayList<ContainerStatus>(), null,
       null, node1ReportForApp));
 
-    Map<ApplicationId, LogAggregationReport> node2ReportForApp =
-        new HashMap<ApplicationId, LogAggregationReport>();
+    List<LogAggregationReport> node2ReportForApp =
+        new ArrayList<LogAggregationReport>();
     String messageForNode2_1 =
         "node2 logAggregation status updated at " + System.currentTimeMillis();
     LogAggregationReport report2 =
-        LogAggregationReport.newInstance(appId, nodeId2,
+        LogAggregationReport.newInstance(appId,
           LogAggregationStatus.RUNNING, messageForNode2_1);
-    node2ReportForApp.put(appId, report2);
+    node2ReportForApp.add(report2);
     node2.handle(new RMNodeStatusEvent(node2.getNodeID(), NodeHealthStatus
       .newInstance(true, null, 0), new ArrayList<ContainerStatus>(), null,
       null, node2ReportForApp));
@@ -205,14 +205,14 @@ public class TestRMAppLogAggregationStatus {
     }
 
     // node1 updates its log aggregation status again
-    Map<ApplicationId, LogAggregationReport> node1ReportForApp2 =
-        new HashMap<ApplicationId, LogAggregationReport>();
+    List<LogAggregationReport> node1ReportForApp2 =
+        new ArrayList<LogAggregationReport>();
     String messageForNode1_2 =
         "node1 logAggregation status updated at " + System.currentTimeMillis();
     LogAggregationReport report1_2 =
-        LogAggregationReport.newInstance(appId, nodeId1,
+        LogAggregationReport.newInstance(appId,
           LogAggregationStatus.RUNNING, messageForNode1_2);
-    node1ReportForApp2.put(appId, report1_2);
+    node1ReportForApp2.add(report1_2);
     node1.handle(new RMNodeStatusEvent(node1.getNodeID(), NodeHealthStatus
       .newInstance(true, null, 0), new ArrayList<ContainerStatus>(), null,
       null, node1ReportForApp2));
@@ -230,8 +230,9 @@ public class TestRMAppLogAggregationStatus {
       if (report.getKey().equals(node1.getNodeID())) {
         Assert.assertEquals(LogAggregationStatus.RUNNING, report.getValue()
           .getLogAggregationStatus());
-        Assert.assertEquals(messageForNode1_1 + messageForNode1_2, report
-          .getValue().getDiagnosticMessage());
+        Assert.assertEquals(
+          messageForNode1_1 + "\n" + messageForNode1_2, report
+            .getValue().getDiagnosticMessage());
       } else if (report.getKey().equals(node2.getNodeID())) {
         Assert.assertEquals(LogAggregationStatus.RUNNING, report.getValue()
           .getLogAggregationStatus());
@@ -268,15 +269,19 @@ public class TestRMAppLogAggregationStatus {
     // Finally, node1 finished its log aggregation and sent out its final
     // log aggregation status. The log aggregation status for node1 should
     // be changed from TIME_OUT to SUCCEEDED
-    Map<ApplicationId, LogAggregationReport> node1ReportForApp3 =
-        new HashMap<ApplicationId, LogAggregationReport>();
-    String messageForNode1_3 =
-        "node1 final logAggregation status updated at "
-            + System.currentTimeMillis();
-    LogAggregationReport report1_3 =
-        LogAggregationReport.newInstance(appId, nodeId1,
-          LogAggregationStatus.SUCCEEDED, messageForNode1_3);
-    node1ReportForApp3.put(appId, report1_3);
+    List<LogAggregationReport> node1ReportForApp3 =
+        new ArrayList<LogAggregationReport>();
+    LogAggregationReport report1_3;
+    for (int i = 0; i < 10 ; i ++) {
+      report1_3 =
+          LogAggregationReport.newInstance(appId,
+            LogAggregationStatus.RUNNING, "test_message_" + i);
+      node1ReportForApp3.add(report1_3);
+    }
+    node1ReportForApp3.add(LogAggregationReport.newInstance(appId,
+      LogAggregationStatus.SUCCEEDED, ""));
+    // For every logAggregationReport cached in memory, we can only save at most
+    // 10 diagnostic messages/failure messages
     node1.handle(new RMNodeStatusEvent(node1.getNodeID(), NodeHealthStatus
       .newInstance(true, null, 0), new ArrayList<ContainerStatus>(), null,
       null, node1ReportForApp3));
@@ -290,8 +295,14 @@ public class TestRMAppLogAggregationStatus {
       if (report.getKey().equals(node1.getNodeID())) {
         Assert.assertEquals(LogAggregationStatus.SUCCEEDED, report.getValue()
           .getLogAggregationStatus());
-        Assert.assertEquals(messageForNode1_1 + messageForNode1_2
-            + messageForNode1_3, report.getValue().getDiagnosticMessage());
+        StringBuilder builder = new StringBuilder();
+        for (int i = 0; i < 9; i ++) {
+          builder.append("test_message_" + i);
+          builder.append("\n");
+        }
+        builder.append("test_message_" + 9);
+        Assert.assertEquals(builder.toString(), report.getValue()
+          .getDiagnosticMessage());
       } else if (report.getKey().equals(node2.getNodeID())) {
         Assert.assertEquals(LogAggregationStatus.TIME_OUT, report.getValue()
           .getLogAggregationStatus());
@@ -301,6 +312,32 @@ public class TestRMAppLogAggregationStatus {
           .fail("should not contain log aggregation report for other nodes");
       }
     }
+
+    // update log aggregationStatus for node2 as FAILED,
+    // so the log aggregation status for the App will become FAILED,
+    // and we only keep the log aggregation reports whose status is FAILED,
+    // so the log aggregation report for node1 will be removed.
+    List<LogAggregationReport> node2ReportForApp2 =
+        new ArrayList<LogAggregationReport>();
+    LogAggregationReport report2_2 =
+        LogAggregationReport.newInstance(appId,
+          LogAggregationStatus.RUNNING_WITH_FAILURE, "Fail_Message");
+    LogAggregationReport report2_3 =
+        LogAggregationReport.newInstance(appId,
+          LogAggregationStatus.FAILED, "");
+    node2ReportForApp2.add(report2_2);
+    node2ReportForApp2.add(report2_3);
+    node2.handle(new RMNodeStatusEvent(node2.getNodeID(), NodeHealthStatus
+      .newInstance(true, null, 0), new ArrayList<ContainerStatus>(), null,
+      null, node2ReportForApp2));
+    Assert.assertEquals(LogAggregationStatus.FAILED,
+      rmApp.getLogAggregationStatusForAppReport());
+    logAggregationStatus = rmApp.getLogAggregationReportsForApp();
+    Assert.assertTrue(logAggregationStatus.size() == 1);
+    Assert.assertTrue(logAggregationStatus.containsKey(node2.getNodeID()));
+    Assert.assertTrue(!logAggregationStatus.containsKey(node1.getNodeID()));
+    Assert.assertEquals("Fail_Message",
+      ((RMAppImpl)rmApp).getLogAggregationFailureMessagesForNM(nodeId2));
   }
 
   @Test (timeout = 10000)
@@ -317,9 +354,11 @@ public class TestRMAppLogAggregationStatus {
     // Enable the log aggregation
     conf.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true);
     rmApp = (RMAppImpl)createRMApp(conf);
-    // If we do not know any NodeManagers for this application ,
-    // the log aggregation status will return null
-    Assert.assertNull(rmApp.getLogAggregationStatusForAppReport());
+    // If we do not know any NodeManagers for this application , and
+    // the log aggregation is enabled, the log aggregation status will
+    // return NOT_START
+    Assert.assertEquals(LogAggregationStatus.NOT_START,
+      rmApp.getLogAggregationStatusForAppReport());
 
     NodeId nodeId1 = NodeId.newInstance("localhost", 1111);
     NodeId nodeId2 = NodeId.newInstance("localhost", 2222);
@@ -329,24 +368,24 @@ public class TestRMAppLogAggregationStatus {
     // If the log aggregation status for all NMs are NOT_START,
     // the log aggregation status for this app will return NOT_START
     rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.NOT_START, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.NOT_START, ""));
     rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.NOT_START, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.NOT_START, ""));
     rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.NOT_START, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.NOT_START, ""));
     rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.NOT_START, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.NOT_START, ""));
     Assert.assertEquals(LogAggregationStatus.NOT_START,
       rmApp.getLogAggregationStatusForAppReport());
 
     rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.NOT_START, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.NOT_START, ""));
     rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.RUNNING, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.RUNNING, ""));
     rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, ""));
     rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.FAILED, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, ""));
     Assert.assertEquals(LogAggregationStatus.RUNNING,
       rmApp.getLogAggregationStatusForAppReport());
 
@@ -357,13 +396,13 @@ public class TestRMAppLogAggregationStatus {
     // others are SUCCEEDED, the log aggregation status for this app will
     // return TIME_OUT
     rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, ""));
     rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.TIME_OUT, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.TIME_OUT, ""));
     rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, ""));
     rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, ""));
     Assert.assertEquals(LogAggregationStatus.TIME_OUT,
       rmApp.getLogAggregationStatusForAppReport());
 
@@ -371,17 +410,59 @@ public class TestRMAppLogAggregationStatus {
     // is at the final state, the log aggregation status for this app will
     // return SUCCEEDED
     rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, ""));
     rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, ""));
     rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, ""));
     rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, ""));
     Assert.assertEquals(LogAggregationStatus.SUCCEEDED,
       rmApp.getLogAggregationStatusForAppReport());
 
     rmApp = (RMAppImpl)createRMApp(conf);
+    // If the log aggregation status for at least one of NMs are RUNNING,
+    // the log aggregation status for this app will return RUNNING
+    rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), LogAggregationStatus.NOT_START, ""));
+    rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), LogAggregationStatus.RUNNING, ""));
+    rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), LogAggregationStatus.NOT_START, ""));
+    rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), LogAggregationStatus.NOT_START, ""));
+    Assert.assertEquals(LogAggregationStatus.RUNNING,
+      rmApp.getLogAggregationStatusForAppReport());
+
+    // If the log aggregation status for at least one of NMs
+    // are RUNNING_WITH_FAILURE, the log aggregation status
+    // for this app will return RUNNING_WITH_FAILURE
+    rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), LogAggregationStatus.NOT_START, ""));
+    rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), LogAggregationStatus.RUNNING, ""));
+    rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), LogAggregationStatus.NOT_START, ""));
+    rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), LogAggregationStatus.RUNNING_WITH_FAILURE,
+      ""));
+    Assert.assertEquals(LogAggregationStatus.RUNNING_WITH_FAILURE,
+      rmApp.getLogAggregationStatusForAppReport());
+
+    // For node4, the previous log aggregation status is RUNNING_WITH_FAILURE,
+    // it will not be changed even it get a new log aggregation status
+    // as RUNNING
+    rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), LogAggregationStatus.NOT_START, ""));
+    rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), LogAggregationStatus.RUNNING, ""));
+    rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), LogAggregationStatus.NOT_START, ""));
+    rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
+      rmApp.getApplicationId(), LogAggregationStatus.RUNNING, ""));
+    Assert.assertEquals(LogAggregationStatus.RUNNING_WITH_FAILURE,
+      rmApp.getLogAggregationStatusForAppReport());
+
     rmApp.handle(new RMAppEvent(rmApp.getApplicationId(), RMAppEventType.KILL));
     Assert.assertTrue(RMAppImpl.isAppInFinalState(rmApp));
     // If at least of one log aggregation status for one NM is FAILED,
@@ -389,13 +470,13 @@ public class TestRMAppLogAggregationStatus {
     // at the final state, the log aggregation status for this app
     // will return FAILED
     rmApp.aggregateLogReport(nodeId1, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.SUCCEEDED, ""));
     rmApp.aggregateLogReport(nodeId2, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.TIME_OUT, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.TIME_OUT, ""));
     rmApp.aggregateLogReport(nodeId3, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.FAILED, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.FAILED, ""));
     rmApp.aggregateLogReport(nodeId4, LogAggregationReport.newInstance(
-      rmApp.getApplicationId(), nodeId1, LogAggregationStatus.SUCCEEDED, ""));
+      rmApp.getApplicationId(), LogAggregationStatus.FAILED, ""));
     Assert.assertEquals(LogAggregationStatus.FAILED,
       rmApp.getLogAggregationStatusForAppReport());
 


[13/37] hadoop git commit: YARN-3539. Updated timeline server documentation and marked REST APIs evolving. Contributed by Steve Loughran.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
index cb8a5d3..acdd8ff 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
@@ -12,8 +12,8 @@
   limitations under the License. See accompanying LICENSE file.
 -->
 
-YARN Timeline Server
-====================
+The YARN Timeline Server
+========================
 
 * [Overview](#Overview)
     * [Introduction](#Introduction)
@@ -21,111 +21,188 @@ YARN Timeline Server
     * [Timeline Structure](#Timeline_Structure)
 * [Deployment](#Deployment)
     * [Configurations](#Configurations)
-    * [Running Timeline server](#Running_Timeline_server)
+    * [Running the Timeline Server](#Running_Timeline_Server)
     * [Accessing generic-data via command-line](#Accessing_generic-data_via_command-line)
 * [Publishing of application specific data](#Publishing_of_application_specific_data)
+* [Timeline Server REST API](#Timeline_Server_REST_API_v1)
+* [Generic Data REST APIs](#GENERIC_DATA_REST_APIS)
 
-Overview
+<a name="Overview"></a>Overview
 ---------
 
-### Introduction  
+### <a name="Introduction"></a>Introduction
 
- Storage and retrieval of application's current as well as historic information in a generic fashion is solved in YARN through the Timeline Server. This serves two responsibilities:
+The Storage and retrieval of application's current and historic information in a
+generic fashion is addressed in YARN through the Timeline Server. It has two responsibilities:
 
-#### Application specific information
+#### Persisting Application Specific Information
 
-  Supports collection of information completely specific to an application or framework. For example, Hadoop MapReduce framework can include pieces of information like number of map tasks, reduce tasks, counters etc. Application developers can publish the specific information to the Timeline server via TimelineClient, the ApplicationMaster and/or the application's containers. This information is then queryable via REST APIs for rendering by application/framework specific UIs.
+The collection and retrieval of information completely specific to an application or framework.
+For example, the Hadoop MapReduce framework can include pieces of information like number of
+map tasks, reduce tasks, counters...etc.
+Application developers can publish the specific information to the Timeline server via `TimelineClient`
+in the Application Master and/or the application's containers.
 
-#### Generic information about completed applications
-  
-  Previously this was done by Application History Server but with  timeline server its just one use case of Timeline server functionality. Generic information includes application level data like queue-name, user information etc in the ApplicationSubmissionContext, list of application-attempts that ran for an application, information about each application-attempt, list of containers run under each application-attempt, and information about each container. Generic data is published by ResourceManager to the timeline store and used by the web-UI to display information about completed applications.
- 
+This information is then queryable via REST APIs for rendering by application/framework specific UIs.
 
-### Current Status
+#### Persisting Generic Information about Completed Applications
 
-  The essential functionality of the timeline server have been completed and it can work in both secure and non secure modes. The generic history service is also built on timeline store. In subsequent releases we will be rolling out next generation timeline service which is scalable and reliable. Currently, Application specific information is only available via RESTful APIs using JSON type content. The ability to install framework specific UIs in YARN is not supported yet.
+Previously this was supported purely for MapReduce jobs by the Application History Server.
+With the introduction of the timeline server, the Application History Server becomes just one use of
+the Timeline Server.
 
-### Timeline Structure
+Generic information includes application level data such as 
+* queue-name, 
+* user information and the like set in the `ApplicationSubmissionContext`,
+* a list of application-attempts that ran for an application
+* information about each application-attempt
+* the list of containers run under each application-attempt
+* information about each container.
 
-![Timeline Structure] (./images/timeline_structure.jpg)
+Generic data is published by the YARN Resource Manager to the timeline store
+ and used by its web-UI to display information about completed applications.
 
-#### TimelineDomain
+### <a name="Current_Status"></a>Current Status and Future Plans
 
-  Domain is like namespace for Timeline server and users can host multiple entities, isolating them from others. Timeline server Security is defined at this level. Domain majorly stores owner info, read & write ACL information, created and modified time stamp information. Domain is uniquely identified by ID.
 
-#### TimelineEntity
+Current status
 
-  Entity contains the the meta information of some conceptual entity and its related events. The entity can be an application, an application attempt, a container or whatever the user-defined object. It contains Primary filters which will be used to index the entities in TimelineStore, such that users should carefully choose the information they want to store as the primary filters. The remaining data can be stored as other information. Entity is uniquely identified by EntityId and EntityType.
+1. The core functionality of the timeline server has been completed.
+1. It works in both secure and non secure clusters.
+1. The generic history service is built on the timeline store.
+1. The history can be stored in memory or in a leveldb database store;
+  the latter ensures the history is preserved over Timeline Server restarts.
+1. The ability to install framework specific UIs in YARN is not supported.
+1. Application specific information is only available via RESTful APIs using JSON type content.
+1. The "Timeline Server v1" REST API has been declared one of the REST APIs
+  whose compatibility will be maintained in future releases.
+1. The single-server implementation of the Timeline Server places a limit on
+  the scalability of the service; it also prevents the service being 
+  High-Availability component of the YARN infrastructure.
 
-#### TimelineEvent
+Future Plans
 
-  TimelineEvent contains the information of an event that is related to some conceptual entity of an application. Users are free to define what the event means, such as starting an application, getting allocated a container and etc.
+1. Future releases will introduce a next generation timeline service
+which is scalable and reliable, "Timeline Server v2".
+1. The expanded features of this service *may not* be available to
+applications using the Timeline Server v1 REST API. That includes extended
+data structures as well as the ability of the client to failover between Timeline Server instances.
 
-Deployment
-----------
+### <a name="Timeline_Structure"></a>Timeline Structure
 
-###Configurations
+![Timeline Structure](./images/timeline_structure.jpg)
 
-#### Basic Configuration
+#### Timeline Domain
 
-| Configuration Property | Description |
-|:---- |:---- |
-| `yarn.timeline-service.enabled` | Indicate to clients whether Timeline service is enabled or not. If enabled, the TimelineClient library used by end-users will post entities and events to the Timeline server. Defaults to false. |
-| `yarn.resourcemanager.system-metrics-publisher.enabled` | The setting that controls whether yarn system metrics is published on the timeline server or not by RM. Defaults to false. |
-| `yarn.timeline-service.generic-application-history.enabled` | Indicate to clients whether to query generic application data from timeline history-service or not. If not enabled then application data is queried only from Resource Manager. Defaults to false. |
+The Timeline Domain offers a namespace for Timeline server allowing
+users to host multiple entities, isolating them from other users and applications.
+Timeline server Security is defined at this level.
+ 
+A "Domain" primarily stores owner info, read and& write ACL information,
+created and modified time stamp information. Each Domain is identified by an ID which 
+must be unique across all users in the YARN cluster.
+
+#### Timeline Entity
+
+A Timeline Entity contains the the meta information of a conceptual entity
+and its related events.
+
+The entity can be an application, an application attempt, a container or
+any user-defined object.
+
+It contains **Primary filters** which will be used to index the entities
+in the Timeline Store. Accordingly, users/applications should carefully
+choose the information they want to store as the primary filters.
+
+The remaining data can be stored as unindexed information.
+Each Entity is uniquely identified by an `EntityId` and `EntityType`.
 
-#### Advanced configuration
+#### Timeline Events
+
+A Timeline Event describes an event that is related to a specific
+Timeline Entity of an application. 
+
+Users are free to define what an event means —such as starting
+an application, getting allocated a container,
+an operation failures or other information considered relevant to users
+and cluster operators.
+
+
+<a name="Deployment"></a>Deployment
+----------
+
+###<a name="Configurations"></a>Configurations
+
+#### Basic Configuration
 
 | Configuration Property | Description |
 |:---- |:---- |
-| `yarn.timeline-service.ttl-enable` | Enable age off of timeline store data. Defaults to true. |
-| `yarn.timeline-service.ttl-ms` | Time to live for timeline store data in milliseconds. Defaults to 604800000 (7 days). |
-| `yarn.timeline-service.handler-thread-count` | Handler thread count to serve the client RPC requests. Defaults to 10. |
-| `yarn.timeline-service.client.max-retries` | Default maximum number of retires for timeline servive client. Defaults to 30. |
-| `yarn.timeline-service.client.retry-interval-ms` | Default retry time interval for timeline servive client. Defaults to 1000. |
+| `yarn.timeline-service.enabled` | Indicate to clients whether Timeline service is enabled or not. If enabled, the `TimelineClient` library used by applications will post entities and events to the Timeline server. Defaults to `false`. |
+| `yarn.resourcemanager.system-metrics-publisher.enabled` | The setting that controls whether or not YARN system metrics are published on the timeline server by RM. Defaults to `false`. |
+| `yarn.timeline-service.generic-application-history.enabled` | Indicate to clients whether to query generic application data from timeline history-service or not. If not enabled then application data is queried only from Resource Manager. Defaults to `false`. |
 
 #### Timeline store and state store configuration
 
 | Configuration Property | Description |
 |:---- |:---- |
-| `yarn.timeline-service.store-class` | Store class name for timeline store. Defaults to org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore. |
-| `yarn.timeline-service.leveldb-timeline-store.path` | Store file name for leveldb timeline store. Defaults to ${hadoop.tmp.dir}/yarn/timeline. |
-| `yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms` | Length of time to wait between deletion cycles of leveldb timeline store in milliseconds. Defaults to 300000. |
-| `yarn.timeline-service.leveldb-timeline-store.read-cache-size` | Size of read cache for uncompressed blocks for leveldb timeline store in bytes. Defaults to 104857600. |
-| `yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size` | Size of cache for recently read entity start times for leveldb timeline store in number of entities. Defaults to 10000. |
-| `yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size` | Size of cache for recently written entity start times for leveldb timeline store in number of entities. Defaults to 10000. |
-| `yarn.timeline-service.recovery.enabled` | Defaults to false. |
-| `yarn.timeline-service.state-store-class` | Store class name for timeline state store. Defaults to org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore. |
+| `yarn.timeline-service.store-class` | Store class name for timeline store. Defaults to `org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore`. |
+| `yarn.timeline-service.leveldb-timeline-store.path` | Store file name for leveldb timeline store. Defaults to `${hadoop.tmp.dir}/yarn/timelin`e. |
+| `yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms` | Length of time to wait between deletion cycles of leveldb timeline store in milliseconds. Defaults to `300000`. |
+| `yarn.timeline-service.leveldb-timeline-store.read-cache-size` | Size of read cache for uncompressed blocks for leveldb timeline store in bytes. Defaults to `104857600`. |
+| `yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size` | Size of cache for recently read entity start times for leveldb timeline store in number of entities. Defaults to `10000`. |
+| `yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size` | Size of cache for recently written entity start times for leveldb timeline store in number of entities. Defaults to `10000`. |
+| `yarn.timeline-service.recovery.enabled` | Defaults to `false`. |
+| `yarn.timeline-service.state-store-class` | Store class name for timeline state store. Defaults to `org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore`. |
 | `yarn.timeline-service.leveldb-state-store.path` | Store file name for leveldb timeline state store. |
 
 #### Web and RPC Configuration
 
 | Configuration Property | Description |
 |:---- |:---- |
-| `yarn.timeline-service.hostname` | The hostname of the Timeline service web application. Defaults to 0.0.0.0. |
-| `yarn.timeline-service.address` | Address for the Timeline server to start the RPC server. Defaults to ${yarn.timeline-service.hostname}:10200. |
-| `yarn.timeline-service.webapp.address` | The http address of the Timeline service web application. Defaults to ${yarn.timeline-service.hostname}:8188. |
-| `yarn.timeline-service.webapp.https.address` | The https address of the Timeline service web application. Defaults to ${yarn.timeline-service.hostname}:8190. |
-| `yarn.timeline-service.bind-host` | The actual address the server will bind to. If this optional address is set, the RPC and webapp servers will bind to this address and the port specified in yarn.timeline-service.address and yarn.timeline-service.webapp.address, respectively. This is most useful for making the service listen to all interfaces by setting to 0.0.0.0. |
-| `yarn.timeline-service.http-cross-origin.enabled` | Enables cross-origin support (CORS) for web services where cross-origin web response headers are needed. For example, javascript making a web services request to the timeline server. Defaults to false. |
+| `yarn.timeline-service.hostname` | The hostname of the Timeline service web application. Defaults to `0.0.0.0` |
+| `yarn.timeline-service.address` | Address for the Timeline server to start the RPC server. Defaults to `${yarn.timeline-service.hostname}:10200`. |
+| `yarn.timeline-service.webapp.address` | The http address of the Timeline service web application. Defaults to `${yarn.timeline-service.hostname}:8188`. |
+| `yarn.timeline-service.webapp.https.address` | The https address of the Timeline service web application. Defaults to `${yarn.timeline-service.hostname}:8190`. |
+| `yarn.timeline-service.bind-host` | The actual address the server will bind to. If this optional address is set, the RPC and webapp servers will bind to this address and the port specified in `yarn.timeline-service.address` and `yarn.timeline-service.webapp.address`, respectively. This is most useful for making the service listen on all interfaces by setting to `0.0.0.0`. |
+| `yarn.timeline-service.http-cross-origin.enabled` | Enables cross-origin support (CORS) for web services where cross-origin web response headers are needed. For example, javascript making a web services request to the timeline server. Defaults to `false`. |
 | `yarn.timeline-service.http-cross-origin.allowed-origins` | Comma separated list of origins that are allowed for web services needing cross-origin (CORS) support. Wildcards `(*)` and patterns allowed. Defaults to `*`. |
-| yarn.timeline-service.http-cross-origin.allowed-methods | Comma separated list of methods that are allowed for web services needing cross-origin (CORS) support. Defaults to GET,POST,HEAD. |
-| `yarn.timeline-service.http-cross-origin.allowed-headers` | Comma separated list of headers that are allowed for web services needing cross-origin (CORS) support. Defaults to X-Requested-With,Content-Type,Accept,Origin. |
-| `yarn.timeline-service.http-cross-origin.max-age` | The number of seconds a pre-flighted request can be cached for web services needing cross-origin (CORS) support. Defaults to 1800. |
+| `yarn.timeline-service.http-cross-origin.allowed-methods` | Comma separated list of methods that are allowed for web services needing cross-origin (CORS) support. Defaults to `GET,POST,HEAD`. |
+| `yarn.timeline-service.http-cross-origin.allowed-headers` | Comma separated list of headers that are allowed for web services needing cross-origin (CORS) support. Defaults to `X-Requested-With,Content-Type,Accept,Origin`. |
+| `yarn.timeline-service.http-cross-origin.max-age` | The number of seconds a pre-flighted request can be cached for web services needing cross-origin (CORS) support. Defaults to `1800`. |
+
+Note that the selection between the HTTP and HTTPS binding is made in the `TimelineClient` based
+upon the value of the YARN-wide configuration option `yarn.http.policy`; the HTTPS endpoint will be
+selected if this policy is either of `HTTPS_ONLY` or `HTTP_AND_HTTPS`.
+
+#### Advanced Server-side configuration
+
+| Configuration Property | Description |
+|:---- |:---- |
+| `yarn.timeline-service.ttl-enable` | Enable deletion of aged data within the timeline store. Defaults to `true`. |
+| `yarn.timeline-service.ttl-ms` | Time to live for timeline store data in milliseconds. Defaults to `604800000` (7 days). |
+| `yarn.timeline-service.handler-thread-count` | Handler thread count to serve the client RPC requests. Defaults to `10`. |
+| `yarn.timeline-service.client.max-retries` | The maximum number of retries for attempts to publish data to the timeline service.Defaults to `30`. |
+| `yarn.timeline-service.client.retry-interval-ms` | The interval in milliseconds between retries for the timeline service client. Defaults to `1000`. |
+
+
 
 #### Security Configuration
 
- Security can be enabled by setting yarn.timeline-service.http-authentication.type to kerberos and further following configurations can be done.
+
+Security can be enabled by setting `yarn.timeline-service.http-authentication.type`
+to `kerberos`, after which the following configuration options are available:
+
 
 | Configuration Property | Description |
 |:---- |:---- |
-| `yarn.timeline-service.http-authentication.type` | Defines authentication used for the timeline server HTTP endpoint. Supported values are: simple / kerberos / #AUTHENTICATION_HANDLER_CLASSNAME#. Defaults to simple. |
-| `yarn.timeline-service.http-authentication.simple.anonymous.allowed` | Indicates if anonymous requests are allowed by the timeline server when using 'simple' authentication. Defaults to true. |
+| `yarn.timeline-service.http-authentication.type` | Defines authentication used for the timeline server HTTP endpoint. Supported values are: `simple` / `kerberos` / #AUTHENTICATION_HANDLER_CLASSNAME#. Defaults to `simple`. |
+| `yarn.timeline-service.http-authentication.simple.anonymous.allowed` | Indicates if anonymous requests are allowed by the timeline server when using 'simple' authentication. Defaults to `true`. |
 | `yarn.timeline-service.principal` | The Kerberos principal for the timeline server. |
-| yarn.timeline-service.keytab | The Kerberos keytab for the timeline server. Defaults to /etc/krb5.keytab. |
-| `yarn.timeline-service.delegation.key.update-interval` | Defaults to 86400000 (1 day). |
-| `yarn.timeline-service.delegation.token.renew-interval` | Defaults to 86400000 (1 day). |
-| `yarn.timeline-service.delegation.token.max-lifetime` | Defaults to 604800000 (7 day). |
+| `yarn.timeline-service.keytab` | The Kerberos keytab for the timeline server. Defaults on Unix to to `/etc/krb5.keytab`. |
+| `yarn.timeline-service.delegation.key.update-interval` | Defaults to `86400000` (1 day). |
+| `yarn.timeline-service.delegation.token.renew-interval` | Defaults to `86400000` (1 day). |
+| `yarn.timeline-service.delegation.token.max-lifetime` | Defaults to `604800000` (7 days). |
+| `yarn.timeline-service.best-effort` | Should the failure to obtain a delegation token be considered an application failure (option = false),  or should the client attempt to continue to publish information without it (option=true). Default: `false` |
 
 #### Enabling the timeline service and the generic history service
 
@@ -156,82 +233,1617 @@ Deployment
 </property>
 ```
 
-### Running Timeline server
+###<a name="Running_Timeline_Server"></a> Running the Timeline Server
 
-  Assuming all the aforementioned configurations are set properly, admins can start the Timeline server/history service with the following command:
+Assuming all the aforementioned configurations are set properly
+admins can start the Timeline server/history service with the following command:
 
-```
-  $ yarn timelineserver
-```
+    yarn timelineserver
 
-  Or users can start the Timeline server / history service as a daemon:
+To start the Timeline server / history service as a daemon, the command is
 
-```
-  $ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh start timelineserver
-```
+    $HADOOP_YARN_HOME/sbin/yarn-daemon.sh start timelineserver
 
-### Accessing generic-data via command-line
+###<a name="Accessing_generic-data_via_command-line"></a> Accessing generic-data via command-line
 
-  Users can access applications' generic historic data via the command line as below. Note that the same commands are usable to obtain the corresponding information about running applications.
+Users can access applications' generic historic data via the command line below
 
-```
-  $ yarn application -status <Application ID>
-  $ yarn applicationattempt -list <Application ID>
-  $ yarn applicationattempt -status <Application Attempt ID>
-  $ yarn container -list <Application Attempt ID>
-  $ yarn container -status <Container ID>
-```
+    $ yarn application -status <Application ID>
+    $ yarn applicationattempt -list <Application ID>
+    $ yarn applicationattempt -status <Application Attempt ID>
+    $ yarn container -list <Application Attempt ID>
+    $ yarn container -status <Container ID>
 
-Publishing of application specific data
-------------------------------------------------
+Note that the same commands are usable to obtain the corresponding information about running applications.
 
-  Developers can define what information they want to record for their applications by composing `TimelineEntity` and  `TimelineEvent` objects, and put the entities and events to the Timeline server via `TimelineClient`. Below is an example:
+###<a name="Publishing_of_application_specific_data"></a> Publishing application specific data
 
-```
-  // Create and start the Timeline client
-  TimelineClient client = TimelineClient.createTimelineClient();
-  client.init(conf);
-  client.start();
+Developers can define what information they want to record for
+their applications by constructing `TimelineEntity` and  `TimelineEvent` objects
+then publishing the entities and events to the Timeline Server
+via the `TimelineClient` API.
+
+Here is an example:
+
+
+    // Create and start the Timeline client
+    TimelineClient client = TimelineClient.createTimelineClient();
+    client.init(conf);
+    client.start();
+
+    try {
+      TimelineDomain myDomain = new TimelineDomain();
+      myDomain.setID("MyDomain");
+      // Compose other Domain info ....
+
+      client.putDomain(myDomain);
+
+      TimelineEntity myEntity = new TimelineEntity();
+      myEntity.setDomainId(myDomain.getId());
+      myEntity.setEntityType("APPLICATION");
+      myEntity.setEntityID("MyApp1")
+      // Compose other entity info
+
+      TimelinePutResponse response = client.putEntities(entity);
+
+      TimelineEvent event = new TimelineEvent();
+      event.setEventType("APP_FINISHED");
+      event.setTimestamp(System.currentTimeMillis());
+      event.addEventInfo("Exit Status", "SUCCESS");
+      // Compose other Event info ....
+
+      myEntity.addEvent(event);
+      TimelinePutResponse response = timelineClient.putEntities(entity);
+
+    } catch (IOException e) {
+      // Handle the exception
+    } catch (RuntimeException e) {
+      // In Hadoop 2.6, if attempts submit information to the Timeline Server fail more than the retry limit,
+      // a RuntimeException will be raised. This may change in future releases, being
+      // replaced with a IOException that is (or wraps) that which triggered retry failures.
+    } catch (YarnException e) {
+      // Handle the exception
+    } finally {
+      // Stop the Timeline client
+      client.stop();
+    }
+
+
+1. Publishing of data to Timeline Server is a synchronous operation; the call will not return until successful.
+1. The `TimelineClient` implementation class is a subclass of the YARN `Service` API; it can be placed
+under a `CompositeService` to ease its lifecycle management.
+1. The result of a `putEntities()` call is a `TimelinePutResponse` object. This contains a
+(hopefully empty) list of those timeline entities reject by the timeline server, along with
+an error code indicating the cause of each failure.
+
+In Hadoop 2.6 and 2.7, the error codes are:
+
+| Error Code | Description |
+|:---- |:---- |
+|1 | No start time |
+|2 | IOException |
+|3 | System Filter conflict (reserved filter key used) |
+|4 | Access Denied |
+|5 | No domain |
+|6 | Forbidden relation |
+
+Further error codes may be defined in future.
+
+**Note** : Following are the points which need to be observed when updating a entity.
+
+* Domain ID should not be modified for already existing entity.
+* After a modification of a Primary filter value, the new value
+will be appended to the old value; the original value will not be replaced.
+* It's advisable to have same primary filters for all updates on entity.
+Any on modification of a primary filter by in an update will result in
+queries with updated primary filter to not fetching the information before the update
+
+# Generic Data Web UI
+
+Users can access the generic historic information of applications via web UI:
+
+    http(s)://<timeline server http(s) address:port>/applicationhistory
+
+## <a name="Timeline_Server_REST_API_v1"></a>Timeline Server REST API V1
+
+Querying the timeline server is currently only supported via REST API calls; there is no API
+client implemented in the YARN libraries. In Java, the Jersey client is effective at querying
+the server, even in secure mode (provided the caller has the appropriate Kerberos tokens or keytab).
+
+The v1 REST API is implemented at under the path, `/ws/v1/timeline/` on the Timeline Server web service.
+
+Here is a non-normative description of the API.
+
+### Root path
+
+    GET /ws/v1/timeline/
+
+Returns a JSON object describing the server instance.
+
+     {"About":"Timeline API"}
+
+
+## <a name="REST_API_DOMAINS"></a>Domains `/ws/v1/timeline/domain`
+
+### Domain summary information `/ws/v1/timeline/domain`
+
+
+    GET /ws/v1/timeline/domain?owner=$OWNER
+
+Returns a list of domains belonging to a specific user, in
+the JSON-marshalled `TimelineDomains` data structure.
+
+The `owner` MUST be set on a GET which is not authenticated.
+
+On an authenticated request, the `owner` defaults to
+the caller.
+
+    PUT /ws/v1/timeline/domain
+
+A PUT of a serialized `TimelineDomain` structure to this path will
+add the domain to the list of domains owned by the specified/current
+user. A successful operation returns status code of 200 and
+a `TimelinePutResponse` containing no errors.
+
+
+### Specific information about a Domain `/ws/v1/timeline/domain/{domainId}`
+
+Returns a JSON-marshalled `TimelineDomain` structure
+describing a domain.
+
+If the domain is not found, then an HTTP 404 response is returned.
+
+
+### POST new domain `/ws/v1/timeline/domain`
+
+Creates a new timeline domain, or overrides an existing one.
+
+When attempting to create a new domain, the ID in the submission MUST
+be unique across all domains in the cluster.
+
+When attempting to update an existing domain, the ID of that domain
+must be set. The submitter must have the appropriate permissions
+to update the domain.
+
+submission: `TimelineDomain`
+
+response: `TimelinePutResponse`
+
+### List domains of a user: GET `/ws/v1/timeline/domain`
+
+Retrieves a list of all domains of a user. 
+
+If an owner is specified, that owner name overrides that of the caller.
+
+| Query Parameter | Description |
+|:---- |:---- |
+| `owner`| owner of the domains to list|
+
+
+    GET http://localhost:8188/ws/v1/timeline/domain?owner=alice
+
+    {
+    "domains":
+      [
+        {
+        "id":"DS_DOMAIN_2",
+        "owner":"alice",
+        "readers":"peter",
+        "writers":"john",
+        "createdtime":1430425000337,
+        "modifiedtime":1430425000337
+        },
+        {
+        "id":"DS_DOMAIN_1",
+        "owner":"alice",
+        "readers":"bar",
+        "writers":"foo",
+        "createdtime":1430424955963,
+        "modifiedtime":1430424955963
+        }
+        ,
+        {"id":"DEFAULT",
+        "description":"System Default Domain",
+        "owner":"alice",
+        "readers":"*",
+        "writers":"*",
+        "createdtime":1430424022699,
+        "modifiedtime":1430424022699
+        }
+      ]
+    }
+
+response: `TimelineDomains`
+
+If the user lacks the permission to list the domains of the specified
+owner, an `TimelineDomains` response with no domain listings is
+returned.
+
+#### Retrieve details of a specific domain: GET `/ws/v1/timeline/domain/{domainId}`
+
+Retrieves the details of a single domain
+
+    GET http://localhost:8188/ws/v1/timeline/domain/DS_DOMAIN_1
+
+Response: `TimelineDomain`
+
+    {
+      "id":"DS_DOMAIN_1",
+      "owner":"zshen",
+      "readers":"bar",
+      "writers":"foo",
+      "createdtime":1430424955963,
+      "modifiedtime":1430424955963
+    }
+
+
+If the user lacks the permission to query the details of that domain,
+a 404, not found exception is returned —the same response which
+is returned if there is no entry with that ID.
+
+
+## <a name="REST_API_POST_TIMELINE_ENTITIES"></a> Posting Timeline Entities
+
+With the Posting Entities API, you can post the entities and events, which
+contain the per-framework information you want to record, to the timeline
+server.
+
+### URI:
+
+    http(s)://<timeline server http(s) address:port>/ws/v1/timeline
+
+### HTTP Operations Supported:
+
+    POST
+
+### Query Parameters Supported:
+
+None
+
+### response: `TimelinePutResponse`
+
+
+### Request Examples:
+
+#### JSON request
+
+HTTP Request:
+
+    POST http://<timeline server http address:port>/ws/v1/timeline
+
+Request Header:
+
+    POST /ws/v1/timeline HTTP/1.1
+    Accept: application/json
+    Content-Type: application/json
+    Transfer-Encoding: chunked
+
+Request Body:
+
+    {
+      "entities" : [ {
+        "entity" : "entity id 0",
+        "entitytype" : "entity type 0",
+        "relatedentities" : {
+          "test ref type 2" : [ "test ref id 2" ],
+          "test ref type 1" : [ "test ref id 1" ]
+        },
+        "events" : [ {
+          "timestamp" : 1395818851590,
+          "eventtype" : "event type 0",
+          "eventinfo" : {
+            "key2" : "val2",
+            "key1" : "val1"
+          }
+        }, {
+          "timestamp" : 1395818851590,
+          "eventtype" : "event type 1",
+          "eventinfo" : {
+            "key2" : "val2",
+            "key1" : "val1"
+          }
+        } ],
+        "primaryfilters" : {
+          "pkey2" : [ "pval2" ],
+          "pkey1" : [ "pval1" ]
+        },
+        "otherinfo" : {
+          "okey2" : "oval2",
+          "okey1" : "oval1"
+        },
+        "starttime" : 1395818851588
+      }, {
+        "entity" : "entity id 1",
+        "entitytype" : "entity type 0",
+        "relatedentities" : {
+          "test ref type 2" : [ "test ref id 2" ],
+          "test ref type 1" : [ "test ref id 1" ]
+        },
+        "events" : [ {
+          "timestamp" : 1395818851590,
+          "eventtype" : "event type 0",
+          "eventinfo" : {
+            "key2" : "val2",
+            "key1" : "val1"
+          }
+        }, {
+          "timestamp" : 1395818851590,
+          "eventtype" : "event type 1",
+          "eventinfo" : {
+            "key2" : "val2",
+            "key1" : "val1"
+          }
+        } ],
+        "primaryfilters" : {
+          "pkey2" : [ "pval2" ],
+          "pkey1" : [ "pval1" ]
+        },
+        "otherinfo" : {
+          "okey2" : "oval2",
+          "okey1" : "oval1"
+        },
+        "starttime" : 1395818851590
+      } ]
+    }
+
+Required fields
+
+Entity: `type` and `id`. `starttime` is required unless the
+entity contains one or more event). 
+Event: `type` and `timestamp`. 
+
+## <a name="REST_API_LIST_TIMELINE_ENTITIES"></a>Timeline Entity List
+
+With the Timeline Entity List API,  you can retrieve a list of entity object,
+sorted by the starting timestamp for the entity, descending. The starting
+timestamp of an entity can be a timestamp specified by the your application.
+If it is not explicitly specified, it will be chosen by the store to be the
+earliest timestamp of the events received in the first post for the entity.
+
+### URI:
+
+Use the following URI to obtain all the entity objects of a given
+`entityType`.
+
+    http(s)://<timeline server http(s) address:port>/ws/v1/timeline/{entityType}
+
+### HTTP Operations Supported:
+
+    GET  http://localhost:8188/ws/v1/timeline/DS_APP_ATTEMPT
+
+
+### Query Parameters Supported:
+
+1. `limit` - A limit on the number of entities to return. If null, defaults to 100.
+1. `windowStart` - The earliest start timestamp to retrieve (exclusive). If
+  null, defaults to retrieving all entities until the limit is reached.
+1. `windowEnd` - The latest start timestamp to retrieve (inclusive). If null,
+  defaults to the max value of Long.
+1. `fromId` - If `fromId` is not null, retrieve entities earlier than and
+  including the specified ID. If no start time is found for the specified ID,
+  an empty list of entities will be returned. The `windowEnd` parameter will take
+  precedence if the start time of this entity falls later than `windowEnd`.
+1. `fromTs` - If `fromTs` is not null, ignore entities that were inserted into the
+  store after the given timestamp. The entity's insert timestamp used for this
+  comparison is the store's system time when the first put for the entity was
+  received (not the entity's start time).
+1. `primaryFilter` - Retrieves only entities that have the specified primary
+  filter. If null, retrieves all entities. This is an indexed retrieval, and no
+  entities that do not match the filter are scanned.
+1. `secondaryFilters` - Retrieves only entities that have exact matches for all
+  the specified filters in their primary filters or other info. This is not an
+  indexed retrieval, so all entities are scanned but only those matching the
+  filters are returned.
+1. fields - Specifies which fields of the entity object to retrieve:
+  `EVENTS`, `RELATED_ENTITIES`, `PRIMARY_FILTERS`, `OTHER_INFO`, `LAST_EVENT_ONLY`. If
+  the set of fields contains `LAST_EVENT_ONLY` and not `EVENTS`, the most recent
+  event for each entity is retrieved. If null, retrieves all fields.
+
+### Elements of the `entities` (Timeline Entity List) Object
+
+When you make a request for the list of timeline entities, the information
+will be returned as a collection of container objects. See also
+{{Timeline Entity}} for syntax of the timeline entity object.
+
+| Item | Data Type | Description|
+|:---- |:---- |:---- |
+| `entities` | array of timeline entity objects(JSON) | The collection of timeline entity objects |
+
+### Response Examples:
+
+#### JSON response
+
+HTTP Request:
+
+    GET http://<timeline server http address:port>/ws/v1/timeline/{entity-type}
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/json
+    Transfer-Encoding: chunked
+
+Response Body:
+
+
+    {
+      "entities":[
+      {
+      "entitytype":"DS_APP_ATTEMPT",
+      "entity":"appattempt_1430424020775_0004_000001",
+      "events":[
+        {
+        "timestamp":1430425008796,
+        "eventtype":"DS_APP_ATTEMPT_END",
+        "eventinfo": { }
+        }
+        {
+        "timestamp":1430425004161,
+        "eventtype":"DS_APP_ATTEMPT_START",
+        "eventinfo": { }
+        }
+      ]
+      "starttime":1430425004161,
+      "domain":"DS_DOMAIN_2",
+      "relatedentities": { },
+      "primaryfilters":
+        {
+        "user":["zshen"]
+        },
+      "otherinfo": { }
+      }
+      {
+      "entitytype":"DS_APP_ATTEMPT",
+      "entity":"appattempt_1430424020775_0003_000001",
+      "starttime":1430424959169,
+      "domain":"DS_DOMAIN_1",
+      "events":[
+        {
+        "timestamp":1430424963836,
+        "eventinfo": { }
+         }
+        {
+        "timestamp":1430424959169,
+        "eventinfo": { }
+        }
+       ]
+      "relatedentities": { },
+      "primaryfilters": {
+        "user":["zshen"]
+       },
+      "otherinfo": { }
+       }
+      ]
+    }
+
+
+## <a name="REST_API_GET_TIMELINE_ENTITY"></a>Timeline Entity
+
+With the Timeline Entity API, you can retrieve the entity information for a
+given entity identifier.
+
+### URI:
+
+Use the following URI to obtain the entity object identified by the
+`entityType` value and the `entityId` value.
+
+    http(s)://<timeline server http(s) address:port>/ws/v1/timeline/{entityType}/{entityId}
+
+### HTTP Operations Supported:
+
+    GET
+
+### Query Parameters Supported:
+
+1. fields - Specifies which fields of the entity object to retrieve:
+  `EVENTS`, `RELATED_ENTITIES`, `PRIMARY_FILTERS`, `OTHER_INFO`, `LAST_EVENT_ONLY`. If
+  the set of fields contains LAST_EVENT_ONLY and not EVENTS, the most recent
+  event for each entity is retrieved. If null, retrieves all fields.
+
+### Elements of the `entity` (Timeline Entity) Object:
+
+See also {{Timeline Event List}} for syntax of the timeline event object.
+
+
+| Item | Data Type | Description|
+|:---- |:---- |:---- |
+| `entity` | string  | The entity id |
+| `entitytype` | string | The entity type |
+| `relatedentities` | map | The related entities' identifiers, which are organized in a map of entityType : [entity1, entity2, ...] |
+| `events` | list | The events of the entity |
+| `primaryfilters` | map | The primary filters of the entity, which are orgainzied in a map of key : [value1, value2, ...] |
+| `otherinfo` | map | The other information of the entity, which is orgainzied in a map of key : value |
+| `starttime` | long | The start time of the entity |
+
+### Response Examples:
+
+#### JSON response
+
+HTTP Request:
+
+    GET http://<timeline server http address:port>/ws/v1/timeline/{entity-type}/{entity-id}
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/json
+    Transfer-Encoding: chunked
+
+Response Body:
+
+    http://localhost:8188/ws/v1/timeline/DS_APP_ATTEMPT/appattempt_1430424020775_0003_000001
+
+    {
+      "events":[
+        {
+        "timestamp":1430424959169,
+        "eventtype":"DS_APP_ATTEMPT_START",
+        "eventinfo":  {}}],
+        "entitytype":"DS_APP_ATTEMPT",
+        "entity":"appattempt_1430424020775_0003_000001",
+        "starttime":1430424959169,
+        "domain":"DS_DOMAIN_1",
+        "relatedentities":  {},
+        "primaryfilters":  {
+            "user":["zshen"]
+            },
+        "otherinfo":  {}
+        }
+      ]
+    }
+  
+
+
+
+## <a name="REST_API_LIST_TIMELINE_EVENTS"></a>Timeline Event List
+
+With the Timeline Events API, you can retrieve the event objects for a list
+of entities all of the same entity type. The events for each entity are
+sorted in order of their timestamps, descending.
+
+### URI:
+
+Use the following URI to obtain the event objects of the given `entityType`.
+
+    http(s)://<timeline server http(s) address:port>/ws/v1/timeline/{entityType}/events
+
+### HTTP Operations Supported:
+
+    GET
+
+### Query Parameters Supported:
+
+1. `entityIds` - The entity IDs to retrieve events for.
+1. `limit` - A limit on the number of events to return for each entity. If null,
+  defaults to 100 events per entity.
+1. `windowStart` - If not null, retrieves only events later than the given time
+  (exclusive)
+1. `windowEnd` - If not null, retrieves only events earlier than the given time
+  (inclusive)
+1. `eventTypes` - Restricts the events returned to the given types. If null,
+  events of all types will be returned.
+
+### Elements of the `events` (Timeline Entity List) Object
+
+When you make a request for the list of timeline events, the information
+will be returned as a collection of event objects.
+
+| Item | Data Type | Description|
+|:---- |:---- |:---- |
+| `events` | array of timeline event objects(JSON) | The collection of timeline event objects |
+
+Below is the elements of a single event object.
+
+
+| Item | Data Type | Description|
+|:---- |:---- |:---- |
+| `eventtype` | string  | The event type |
+| `eventinfo` | map | The information of the event, which is orgainzied in a map of `key` : `value` |
+| `timestamp` | long | The timestamp of the event |
+
+### Response Examples:
 
-  try {
-    TimelineDomain myDomain = new TimelineDomain();
-    myDomain.setID("MyDomain");
-    // Compose other Domain info ....
+#### JSON response
 
-    client.putDomain(myDomain);
+HTTP Request:
 
-    TimelineEntity myEntity = new TimelineEntity();
-    myEntity.setDomainId(myDomain.getId());
-    myEntity.setEntityType("APPLICATION");
-    myEntity.setEntityID("MyApp1")
-    // Compose other entity info
+    GET http://<timeline server http address:port>/ws/v1/timeline/entity%20type%200/events
 
-    TimelinePutResponse response = client.putEntities(entity);
+Response Header:
 
+    HTTP/1.1 200 OK
+    Content-Type: application/json
+    Transfer-Encoding: chunked
+
+Response Body:
+
+
+    GET http://localhost:8188/ws/v1/timeline/DS_APP_ATTEMPT/events?entityId=appattempt_1430424020775_0003_000001
     
-    TimelineEvent event = new TimelineEvent();
-    event.setEventType("APP_FINISHED");
-    event.setTimestamp(System.currentTimeMillis());
-    event.addEventInfo("Exit Status", "SUCCESS");
-    // Compose other Event info ....
-
-    myEntity.addEvent(event);
-    timelineClient.putEntities(entity);
-
-  } catch (IOException e) {
-    // Handle the exception
-  } catch (YarnException e) {
-    // Handle the exception
-  }
-
-  // Stop the Timeline client
-  client.stop();
-```
+    
+    {
+    "events": [
+      {
+      "entity":"appattempt_1430424020775_0003_000001",
+      "entitytype":"DS_APP_ATTEMPT"}
+      "events":[
+        {
+        "timestamp":1430424963836,
+        "eventtype":"DS_APP_ATTEMPT_END",
+        "eventinfo":{}},
+        {
+        "timestamp":1430424959169,
+        "eventtype":"DS_APP_ATTEMPT_START",
+        "eventinfo":{}}
+        ],
+       }
+      ]
+    }
+
+
+
+
+
+#<a name="GENERIC_DATA_REST_APIS"></a> Generic Data REST APIs
+
+Users can access the generic historic information of applications via REST
+APIs.
+
+## <a name="REST_API_LIST_APPLICATIONS"></a>Application List
+
+With the Application List API, you can obtain a collection of resources, each
+of which represents an application. When you run a GET operation on this
+resource, you obtain a collection of application objects.
+
+### URI:
+
+    http(s)://<timeline server http(s) address:port>/ws/v1/applicationhistory/apps
+
+### HTTP Operations Supported:
+
+    GET
+
+### Query Parameters Supported:
+
+1. `states` - applications matching the given application states, specified as a
+comma-separated list
+
+1. `finalStatus` - the final status of the application - reported by the
+application itself
+
+1. `user` - user name
+
+1. `queue` - queue name
+
+1. `limit` - total number of app objects to be returned
+
+1. `startedTimeBegin` - applications with start time beginning with this time,
+specified in ms since epoch
+
+1. `startedTimeEnd` - applications with start time ending with this time,
+specified in ms since epoch
+
+1. `finishedTimeBegin` - applications with finish time beginning with this time,
+specified in ms since epoch
+
+1. `finishedTimeEnd` - applications with finish time ending with this time,
+specified in ms since epoch
+
+1. `applicationTypes` - applications matching the given application types,
+specified as a comma-separated list
+
+### Elements of the `apps` (Application List) Object
+
+When you make a request for the list of applications, the information will be
+returned as a collection of application objects. See also `Application` for
+syntax of the application object.
+
+| Item  | Data Type | Description  |
+|:---- |:---- |:---- |
+| `app` | array of app objects(JSON)/zero or more application objects(XML) | The collection of application objects |
+
+### Response Examples:
+
+#### JSON response
+
+HTTP Request:
+
+    GET http://<timeline server http address:port>/ws/v1/applicationhistory/apps
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/json
+    Transfer-Encoding: chunked
+
+Response Body:
+
+
+    {
+      "app":
+      [
+          {
+          "appId":"application_1430424020775_0004",
+          "currentAppAttemptId":"appattempt_1430424020775_0004_000001",
+          "user":"zshen",
+          "name":"DistributedShell",
+          "queue":"default",
+          "type":"YARN",
+          "host":"d-69-91-129-173.dhcp4.washington.edu/69.91.129.173",
+          "rpcPort":-1,
+          "appState":"FINISHED",
+          "progress":100.0,
+          "diagnosticsInfo":"",
+          "originalTrackingUrl":"N/A",
+          "trackingUrl":"http://d-69-91-129-173.dhcp4.washington.edu:8088/proxy/application_1430424020775_0004/",
+          "finalAppStatus":"SUCCEEDED",
+          "submittedTime":1430425001004,
+          "startedTime":1430425001004,
+          "finishedTime":1430425008861,
+          "elapsedTime":7857},
+          {
+          "appId":"application_1430424020775_0003",
+          "currentAppAttemptId":"appattempt_1430424020775_0003_000001",
+          "user":"zshen",
+          "name":"DistributedShell",
+          "queue":"default",
+          "type":"YARN",
+          "host":"d-69-91-129-173.dhcp4.washington.edu/69.91.129.173",
+          "rpcPort":-1,
+          "appState":"FINISHED",
+          "progress":100.0,
+          "diagnosticsInfo":"",
+          "originalTrackingUrl":"N/A",
+          "trackingUrl":"http://d-69-91-129-173.dhcp4.washington.edu:8088/proxy/application_1430424020775_0003/",
+          "finalAppStatus":"SUCCEEDED",
+          "submittedTime":1430424956650,
+          "startedTime":1430424956650,
+          "finishedTime":1430424963907,
+          "elapsedTime":7257},
+          {
+          "appId":"application_1430424020775_0002",
+          "currentAppAttemptId":"appattempt_1430424020775_0002_000001",
+          "user":"zshen",
+          "name":"DistributedShell",
+          "queue":"default",
+          "type":"YARN",
+          "host":"d-69-91-129-173.dhcp4.washington.edu/69.91.129.173",
+          "rpcPort":-1,
+          "appState":"FINISHED",
+          "progress":100.0,
+          "diagnosticsInfo":"",
+          "originalTrackingUrl":"N/A",
+          "trackingUrl":"http://d-69-91-129-173.dhcp4.washington.edu:8088/proxy/application_1430424020775_0002/",
+          "finalAppStatus":"SUCCEEDED",
+          "submittedTime":1430424769395,
+          "startedTime":1430424769395,
+          "finishedTime":1430424776594,
+          "elapsedTime":7199
+          }
+      ]
+    }
+
+
+
+#### XML response
+
+HTTP Request:
+
+    GET http://localhost:8188/ws/v1/applicationhistory/apps
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/xml
+    Content-Length: 1710
+
+Response Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <apps>
+      <app>
+        <appId>application_1430424020775_0004</appId>
+        <currentAppAttemptId>appattempt_1430424020775_0004_000001</currentAppAttemptId>
+        <user>zshen</user>
+        <name>DistributedShell</name>
+        <queue>default</queue>
+        <type>YARN</type>
+        <host>d-69-91-129-173.dhcp4.washington.edu/69.91.129.173</host>
+        <rpcPort>-1</rpcPort>
+        <appState>FINISHED</appState>
+        <progress>100.0</progress>
+        <diagnosticsInfo></diagnosticsInfo>
+        <originalTrackingUrl>N/A</originalTrackingUrl>
+        <trackingUrl>http://d-69-91-129-173.dhcp4.washington.edu:8088/proxy/application_1430424020775_0004/</trackingUrl>
+        <finalAppStatus>SUCCEEDED</finalAppStatus>
+        <submittedTime>1430425001004</submittedTime>
+        <startedTime>1430425001004</startedTime>
+        <finishedTime>1430425008861</finishedTime>
+        <elapsedTime>7857</elapsedTime>
+      </app>
+      <app>
+        <appId>application_1430424020775_0003</appId>
+        <currentAppAttemptId>appattempt_1430424020775_0003_000001</currentAppAttemptId>
+        <user>zshen</user>
+        <name>DistributedShell</name>
+        <queue>default</queue>
+        <type>YARN</type>
+        <host>d-69-91-129-173.dhcp4.washington.edu/69.91.129.173</host>
+        <rpcPort>-1</rpcPort>
+        <appState>FINISHED</appState>
+        <progress>100.0</progress>
+        <diagnosticsInfo></diagnosticsInfo>
+        <originalTrackingUrl>N/A</originalTrackingUrl>
+        <trackingUrl>http://d-69-91-129-173.dhcp4.washington.edu:8088/proxy/application_1430424020775_0003/</trackingUrl>
+        <finalAppStatus>SUCCEEDED</finalAppStatus>
+        <submittedTime>1430424956650</submittedTime>
+        <startedTime>1430424956650</startedTime>
+        <finishedTime>1430424963907</finishedTime>
+        <elapsedTime>7257</elapsedTime>
+      </app>
+      <app>
+        <appId>application_1430424020775_0002</appId>
+        <currentAppAttemptId>appattempt_1430424020775_0002_000001</currentAppAttemptId>
+        <user>zshen</user>
+        <name>DistributedShell</name>
+        <queue>default</queue>
+        <type>YARN</type>
+        <host>d-69-91-129-173.dhcp4.washington.edu/69.91.129.173</host>
+        <rpcPort>-1</rpcPort>
+        <appState>FINISHED</appState>
+        <progress>100.0</progress>
+        <diagnosticsInfo></diagnosticsInfo>
+        <originalTrackingUrl>N/A</originalTrackingUrl>
+        <trackingUrl>http://d-69-91-129-173.dhcp4.washington.edu:8088/proxy/application_1430424020775_0002/</trackingUrl>
+        <finalAppStatus>SUCCEEDED</finalAppStatus>
+        <submittedTime>1430424769395</submittedTime>
+        <startedTime>1430424769395</startedTime>
+        <finishedTime>1430424776594</finishedTime>
+        <elapsedTime>7199</elapsedTime>
+      </app>
+      <app>
+        <appId>application_1430424020775_0001</appId>
+        <currentAppAttemptId>appattempt_1430424020775_0001_000001</currentAppAttemptId>
+        <user>zshen</user>
+        <name>QuasiMonteCarlo</name>
+        <queue>default</queue>
+        <type>MAPREDUCE</type>
+        <host>localhost</host>
+        <rpcPort>56264</rpcPort>
+        <appState>FINISHED</appState>
+        <progress>100.0</progress>
+        <diagnosticsInfo></diagnosticsInfo>
+        <originalTrackingUrl>http://d-69-91-129-173.dhcp4.washington.edu:19888/jobhistory/job/job_1430424020775_0001</originalTrackingUrl>
+        <trackingUrl>http://d-69-91-129-173.dhcp4.washington.edu:8088/proxy/application_1430424020775_0001/</trackingUrl>
+        <finalAppStatus>SUCCEEDED</finalAppStatus>
+        <submittedTime>1430424053809</submittedTime>
+        <startedTime>1430424053809</startedTime>
+        <finishedTime>1430424072153</finishedTime>
+        <elapsedTime>18344</elapsedTime>
+      </app>
+    </apps>
+
+## <a name="REST_API_GET_APPLICATION"></a>Application
+
+With the Application API, you can get an application resource contains
+information about a particular application that was running on an YARN
+cluster.
+
+It is essentially a JSON-serialized form of the YARN `ApplicationReport` structure.
+
+### URI:
+
+Use the following URI to obtain an application object identified by the `appid` value.
+
+    http(s)://<timeline server http(s) address:port>/ws/v1/applicationhistory/apps/{appid}
+
+### HTTP Operations Supported:
+
+    GET
+
+### Query Parameters Supported:
+
+None
+
+### Elements of the `app` (Application) Object:
+
+| Item         | Data Type   | Description                   |
+|:---- |:----  |:---- |
+| `appId` | string  | The application ID |
+| `user` | string  | The user who started the application |
+| `name` | string  | The application name |
+| `type` | string  | The application type |
+| `queue` | string  | The queue to which the application submitted |
+| `appState` | string | The application state according to the ResourceManager - valid values are members of the YarnApplicationState enum: `FINISHED`, `FAILED`, `KILLED` |
+| `finalStatus` | string | The final status of the application if finished - reported by the application itself - valid values are: `UNDEFINED`, `SUCCEEDED`, `FAILED`, `KILLED` |
+| `progress` | float | The reported progress of the application as a percent. Long-lived YARN services may not provide a meaninful value here —or use it as a metric of actual vs desired container counts |
+| `trackingUrl` | string | The web URL of the application (via the RM Proxy) |
+| `originalTrackingUrl` | string | The actual web URL of the application |
+| `diagnosticsInfo` | string | Detailed diagnostics information on a completed application|
+| `startedTime` | long | The time in which application started (in ms since epoch) |
+| `finishedTime` | long | The time in which the application finished (in ms since epoch) |
+| `elapsedTime` | long | The elapsed time since the application started (in ms) |
+| `allocatedMB` | int | The sum of memory in MB allocated to the application's running containers |
+| `allocatedVCores` | int | The sum of virtual cores allocated to the application's running containers |
+| `currentAppAttemptId` | string | The latest application attempt ID |
+| `host` | string | The host of the ApplicationMaster |
+| `rpcPort` | int | The RPC port of the ApplicationMaster; zero if no IPC service declared. |
+
+### Response Examples:
+
+#### JSON response
+
+HTTP Request:
+
+    http://localhost:8188/ws/v1/applicationhistory/apps/application_1430424020775_0001
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/json
+    Transfer-Encoding: chunked
+
+Response Body:
+
+    {
+      "appId": "application_1430424020775_0001",
+      "currentAppAttemptId": "appattempt_1430424020775_0001_000001",
+      "user": "zshen",
+      "name": "QuasiMonteCarlo",
+      "queue": "default",
+      "type": "MAPREDUCE",
+      "host": "localhost",
+      "rpcPort": 56264,
+      "appState": "FINISHED",
+      "progress": 100.0,
+      "diagnosticsInfo": "",
+      "originalTrackingUrl": "http://d-69-91-129-173.dhcp4.washington.edu:19888/jobhistory/job/job_1430424020775_0001",
+      "trackingUrl": "http://d-69-91-129-173.dhcp4.washington.edu:8088/proxy/application_1430424020775_0001/",
+      "finalAppStatus": "SUCCEEDED",
+      "submittedTime": 1430424053809,
+      "startedTime": 1430424053809,
+      "finishedTime": 1430424072153,
+      "elapsedTime": 18344
+    }
+
+#### XML response
+
+HTTP Request:
+
+    GET http://localhost:8188/ws/v1/applicationhistory/apps/application_1430424020775_0001
+    Accept: application/xml
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/xml
+    Content-Length: 873
+
+Response Body:
+
+     <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+     <app>
+       <appId>application_1430424020775_0001</appId>
+       <currentAppAttemptId>appattempt_1430424020775_0001_000001</currentAppAttemptId>
+       <user>zshen</user>
+       <name>QuasiMonteCarlo</name>
+       <queue>default</queue>
+       <type>MAPREDUCE</type>
+       <host>localhost</host>
+       <rpcPort>56264</rpcPort>
+       <appState>FINISHED</appState>
+       <progress>100.0</progress>
+       <diagnosticsInfo></diagnosticsInfo>
+       <originalTrackingUrl>http://d-69-91-129-173.dhcp4.washington.edu:19888/jobhistory/job/job_1430424020775_0001</originalTrackingUrl>
+       <trackingUrl>http://d-69-91-129-173.dhcp4.washington.edu:8088/proxy/application_1430424020775_0001/</trackingUrl>
+       <finalAppStatus>SUCCEEDED</finalAppStatus>
+       <submittedTime>1430424053809</submittedTime>
+       <startedTime>1430424053809</startedTime>
+       <finishedTime>1430424072153</finishedTime>
+       <elapsedTime>18344</elapsedTime>
+     </app>
+
+## <a name="REST_API_APPLICATION_ATTEMPT_LIST"></a>Application Attempt List
+
+With the Application Attempt List API, you can obtain a collection of
+resources, each of which represents an application attempt. When you run a
+GET operation on this resource, you obtain a collection of application
+attempt objects.
+
+### URI:
+
+Use the following URI to obtain all the attempt objects of an application
+identified by the `appid` value.
+
+    http(s)://<timeline server http(s) address:port>/ws/v1/applicationhistory/apps/{appid}/appattempts
+
+### HTTP Operations Supported:
+
+    GET
+
+### Query Parameters Supported:
+
+None
+
+### Elements of the `appattempts` (Application Attempt List) Object
+
+When you make a request for the list of application attempts, the information
+will be returned as a collection of application attempt objects. See 
+[Application Attempt](#REST_API_APPLICATION_ATTEMPT) for the syntax of 
+the application attempt object.
+
+| Item         | Data Type   | Description                  |
+|:---- |:---- |:---- |
+| `appattempt` | array of appattempt objects(JSON)/zero or more application attempt objects(XML) | The collection of application attempt objects |
+
+### Response Examples:
+
+#### JSON response
+
+HTTP Request:
+
+    GET  http://localhost:8188/ws/v1/applicationhistory/apps/application_1430424020775_0001/appattempts
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/json
+    Transfer-Encoding: chunked
 
-  **Note** : Following are the points which needs to be observed during updating a entity.
+Response Body:
 
-  * Domain ID should not be modified for already existing entity.
+    {
+      "appAttempt": [
+        {
+          "appAttemptId": "appattempt_1430424020775_0001_000001",
+          "host": "localhost",
+          "rpcPort": 56264,
+          "trackingUrl": "http://d-69-91-129-173.dhcp4.washington.edu:8088/proxy/application_1430424020775_0001/",
+          "originalTrackingUrl": "http://d-69-91-129-173.dhcp4.washington.edu:19888/jobhistory/job/job_1430424020775_0001",
+          "diagnosticsInfo": "",
+          "appAttemptState": "FINISHED",
+          "amContainerId": "container_1430424020775_0001_01_000001"
+        }
+      ]
+    }
 
-  * Its advisable to have same primary filters for all updates on entity. As on modification of primary filter by subsequent updates will result in not fetching the information before the update when queried with updated primary filter.
+#### XML response
+
+HTTP Request:
+
+    GET http://localhost:8188/ws/v1/applicationhistory/apps/application_1430424020775_0001/appattempts
+    Accept: application/xml
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/xml
+
+Response Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <appAttempts>
+      <appAttempt>
+        <appAttemptId>appattempt_1430424020775_0001_000001</appAttemptId>
+        <host>localhost</host>
+        <rpcPort>56264</rpcPort>
+        <trackingUrl>http://d-69-91-129-173.dhcp4.washington.edu:8088/proxy/application_1430424020775_0001/</trackingUrl>
+        <originalTrackingUrl>http://d-69-91-129-173.dhcp4.washington.edu:19888/jobhistory/job/job_1430424020775_0001</originalTrackingUrl>
+        <diagnosticsInfo></diagnosticsInfo>
+        <appAttemptState>FINISHED</appAttemptState>
+        <amContainerId>container_1430424020775_0001_01_000001</amContainerId>
+      </appAttempt>
+    </appAttempts>
+
+## <a name="REST_API_APPLICATION_ATTEMPT"></a>Application Attempt
+
+With the Application Attempt API, you can get an application attempt resource
+contains information about a particular application attempt of an application
+that was running on an YARN cluster.
+
+### URI:
+
+Use the following URI to obtain an application attempt object identified by
+the `appid` value and the `appattemptid` value.
+
+    http(s)://<timeline server http(s) address:port>/ws/v1/applicationhistory/apps/{appid}/appattempts/{appattemptid}
+
+### HTTP Operations Supported:
+
+    GET
+
+### Query Parameters Supported:
+
+None
+
+### Elements of the `appattempt` (Application Attempt) Object:
+
+
+| Item | Data Type   | Description  |
+|:---- |:---- |:---- |
+| `appAttemptId` | string  | The application attempt Id |
+| `amContainerId` | string  | The ApplicationMaster container Id |
+| `appAttemptState` | string | The application attempt state according to the ResourceManager - valid values are members of the YarnApplicationAttemptState enum: FINISHED, FAILED, KILLED |
+| `trackingUrl` | string | The web URL that can be used to track the application |
+| `originalTrackingUrl` | string | The actual web URL of the application |
+| `diagnosticsInfo` | string | Detailed diagnostics information |
+| `host` | string | The host of the ApplicationMaster |
+| `rpcPort` | int | The rpc port of the ApplicationMaster |
+
+### Response Examples:
+
+#### JSON response
+
+HTTP Request:
+
+    http://localhost:8188/ws/v1/applicationhistory/apps/application_1430424020775_0001/appattempts/appattempt_1430424020775_0001_000001
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/json
+    Transfer-Encoding: chunked
+
+Response Body:
+
+    {
+      "appAttemptId": "appattempt_1430424020775_0001_000001",
+      "host": "localhost",
+      "rpcPort": 56264,
+      "trackingUrl": "http://d-69-91-129-173.dhcp4.washington.edu:8088/proxy/application_1430424020775_0001/",
+      "originalTrackingUrl": "http://d-69-91-129-173.dhcp4.washington.edu:19888/jobhistory/job/job_1430424020775_0001",
+      "diagnosticsInfo": "",
+      "appAttemptState": "FINISHED",
+      "amContainerId": "container_1430424020775_0001_01_000001"
+    }
+
+
+#### XML response
+
+HTTP Request:
+
+    GET http://<timeline server http address:port>/ws/v1/applicationhistory/apps/application_1395789200506_0001/appattempts/appattempt_1395789200506_0001_000001
+    Accept: application/xml
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/xml
+    Content-Length: 488
+
+Response Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <appAttempt>
+      <appAttemptId>appattempt_1430424020775_0001_000001</appAttemptId>
+      <host>localhost</host>
+      <rpcPort>56264</rpcPort>
+      <trackingUrl>http://d-69-91-129-173.dhcp4.washington.edu:8088/proxy/application_1430424020775_0001/</trackingUrl>
+      <originalTrackingUrl>http://d-69-91-129-173.dhcp4.washington.edu:19888/jobhistory/job/job_1430424020775_0001</originalTrackingUrl>
+      <diagnosticsInfo></diagnosticsInfo>
+      <appAttemptState>FINISHED</appAttemptState>
+      <amContainerId>container_1430424020775_0001_01_000001</amContainerId>
+    </appAttempt>
+
+
+## <a name="REST_API_CONTAINER_LIST"></a>Container List
+
+With the Container List API, you can obtain a collection of resources, each
+of which represents a container. When you run a GET operation on this
+resource, you obtain a collection of container objects.
+
+### URI:
+
+Use the following URI to obtain all the container objects of an application
+attempt identified by the `appid` value and the `appattemptid` value.
+
+    http(s)://<timeline server http(s) address:port>/ws/v1/applicationhistory/apps/{appid}/appattempts/{appattemptid}/containers
+
+
+### HTTP Operations Supported:
+
+    GET
+
+### Query Parameters Supported:
+
+None
+
+### Elements of the `containers` (Container List) Object
+
+When you make a request for the list of containers, the information will be
+returned as a collection of container objects. See also {{Container}} for
+syntax of the container object.
+
+| Item | Data Type   | Description |
+|:---- |:---- |:---- |
+| `container` | array of container objects(JSON)/zero or more container objects(XML) | The collection of container objects |
+
+### Response Examples:
+
+#### JSON response
+
+HTTP Request:
+
+    GET http://localhost:8188/ws/v1/applicationhistory/apps/application_1430424020775_0001/appattempts/appattempt_1430424020775_0001_000001/containers?
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/json
+    Transfer-Encoding: chunked
+
+Response Body:
+
+    {
+      "container": [
+        {
+          "containerId": "container_1430424020775_0001_01_000007",
+          "allocatedMB": 1024,
+          "allocatedVCores": 1,
+          "assignedNodeId": "localhost:9105",
+          "priority": 10,
+          "startedTime": 1430424068296,
+          "finishedTime": 1430424073006,
+          "elapsedTime": 4710,
+          "diagnosticsInfo": "Container killed by the ApplicationMaster.\nContainer killed on request. Exit code is 143\nContainer exited with a non-zero exit code 143\n",
+          "logUrl": "http://0.0.0.0:8188/applicationhistory/logs/localhost:9105/container_1430424020775_0001_01_000007/container_1430424020775_0001_01_000007/zshen",
+          "containerExitStatus": -105,
+          "containerState": "COMPLETE",
+          "nodeHttpAddress": "http://localhost:8042"
+        },
+        {
+          "containerId": "container_1430424020775_0001_01_000006",
+          "allocatedMB": 1024,
+          "allocatedVCores": 1,
+          "assignedNodeId": "localhost:9105",
+          "priority": 20,
+          "startedTime": 1430424060317,
+          "finishedTime": 1430424068293,
+          "elapsedTime": 7976,
+          "diagnosticsInfo": "Container killed by the ApplicationMaster.\nContainer killed on request. Exit code is 143\nContainer exited with a non-zero exit code 143\n",
+          "logUrl": "http://0.0.0.0:8188/applicationhistory/logs/localhost:9105/container_1430424020775_0001_01_000006/container_1430424020775_0001_01_000006/zshen",
+          "containerExitStatus": -105,
+          "containerState": "COMPLETE",
+          "nodeHttpAddress": "http://localhost:8042"
+        },
+        {
+          "containerId": "container_1430424020775_0001_01_000005",
+          "allocatedMB": 1024,
+          "allocatedVCores": 1,
+          "assignedNodeId": "localhost:9105",
+          "priority": 20,
+          "startedTime": 1430424060316,
+          "finishedTime": 1430424068294,
+          "elapsedTime": 7978,
+          "diagnosticsInfo": "Container killed by the ApplicationMaster.\nContainer killed on request. Exit code is 143\nContainer exited with a non-zero exit code 143\n",
+          "logUrl": "http://0.0.0.0:8188/applicationhistory/logs/localhost:9105/container_1430424020775_0001_01_000005/container_1430424020775_0001_01_000005/zshen",
+          "containerExitStatus": -105,
+          "containerState": "COMPLETE",
+          "nodeHttpAddress": "http://localhost:8042"
+        },
+        {
+          "containerId": "container_1430424020775_0001_01_000003",
+          "allocatedMB": 1024,
+          "allocatedVCores": 1,
+          "assignedNodeId": "localhost:9105",
+          "priority": 20,
+          "startedTime": 1430424060315,
+          "finishedTime": 1430424068289,
+          "elapsedTime": 7974,
+          "diagnosticsInfo": "Container killed by the ApplicationMaster.\nContainer killed on request. Exit code is 143\nContainer exited with a non-zero exit code 143\n",
+          "logUrl": "http://0.0.0.0:8188/applicationhistory/logs/localhost:9105/container_1430424020775_0001_01_000003/container_1430424020775_0001_01_000003/zshen",
+          "containerExitStatus": -105,
+          "containerState": "COMPLETE",
+          "nodeHttpAddress": "http://localhost:8042"
+        },
+        {
+          "containerId": "container_1430424020775_0001_01_000004",
+          "allocatedMB": 1024,
+          "allocatedVCores": 1,
+          "assignedNodeId": "localhost:9105",
+          "priority": 20,
+          "startedTime": 1430424060315,
+          "finishedTime": 1430424068291,
+          "elapsedTime": 7976,
+          "diagnosticsInfo": "Container killed by the ApplicationMaster.\nContainer killed on request. Exit code is 143\nContainer exited with a non-zero exit code 143\n",
+          "logUrl": "http://0.0.0.0:8188/applicationhistory/logs/localhost:9105/container_1430424020775_0001_01_000004/container_1430424020775_0001_01_000004/zshen",
+          "containerExitStatus": -105,
+          "containerState": "COMPLETE",
+          "nodeHttpAddress": "http://localhost:8042"
+        },
+        {
+          "containerId": "container_1430424020775_0001_01_000002",
+          "allocatedMB": 1024,
+          "allocatedVCores": 1,
+          "assignedNodeId": "localhost:9105",
+          "priority": 20,
+          "startedTime": 1430424060313,
+          "finishedTime": 1430424067250,
+          "elapsedTime": 6937,
+          "diagnosticsInfo": "Container killed by the ApplicationMaster.\nContainer killed on request. Exit code is 143\nContainer exited with a non-zero exit code 143\n",
+          "logUrl": "http://0.0.0.0:8188/applicationhistory/logs/localhost:9105/container_1430424020775_0001_01_000002/container_1430424020775_0001_01_000002/zshen",
+          "containerExitStatus": -105,
+          "containerState": "COMPLETE",
+          "nodeHttpAddress": "http://localhost:8042"
+        },
+        {
+          "containerId": "container_1430424020775_0001_01_000001",
+          "allocatedMB": 2048,
+          "allocatedVCores": 1,
+          "assignedNodeId": "localhost:9105",
+          "priority": 0,
+          "startedTime": 1430424054314,
+          "finishedTime": 1430424079022,
+          "elapsedTime": 24708,
+          "diagnosticsInfo": "",
+          "logUrl": "http://0.0.0.0:8188/applicationhistory/logs/localhost:9105/container_1430424020775_0001_01_000001/container_1430424020775_0001_01_000001/zshen",
+          "containerExitStatus": 0,
+          "containerState": "COMPLETE",
+          "nodeHttpAddress": "http://localhost:8042"
+        }
+      ]
+    }
+    
+
+#### XML response
+
+HTTP Request:
+
+    GET http://localhost:8188/ws/v1/applicationhistory/apps/application_1430424020775_0001/appattempts/appattempt_1430424020775_0001_000001/containers
+    Accept: application/xml
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Content-Length: 1428
+
+Response Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <containers>
+      <container>
+        <containerId>container_1430424020775_0001_01_000007</containerId>
+        <allocatedMB>1024</allocatedMB>
+        <allocatedVCores>1</allocatedVCores>
+        <assignedNodeId>localhost:9105</assignedNodeId>
+        <priority>10</priority>
+        <startedTime>1430424068296</startedTime>
+        <finishedTime>1430424073006</finishedTime>
+        <elapsedTime>4710</elapsedTime>
+        <diagnosticsInfo>Container killed by the ApplicationMaster.
+          Container killed on request. Exit code is 143
+          Container exited with a non-zero exit code 143
+        </diagnosticsInfo>
+        <logUrl>http://0.0.0.0:8188/applicationhistory/logs/localhost:9105/container_1430424020775_0001_01_000007/container_1430424020775_0001_01_000007/zshen</logUrl>
+        <containerExitStatus>-105</containerExitStatus>
+        <containerState>COMPLETE</containerState>
+        <nodeHttpAddress>http://localhost:8042</nodeHttpAddress>
+      </container>
+      <container>
+        <containerId>container_1430424020775_0001_01_000006</containerId>
+        <allocatedMB>1024</allocatedMB>
+        <allocatedVCores>1</allocatedVCores>
+        <assignedNodeId>localhost:9105</assignedNodeId>
+        <priority>20</priority>
+        <startedTime>1430424060317</startedTime>
+        <finishedTime>1430424068293</finishedTime>
+        <elapsedTime>7976</elapsedTime>
+        <diagnosticsInfo>Container killed by the ApplicationMaster.
+          Container killed on request. Exit code is 143
+          Container exited with a non-zero exit code 143
+        </diagnosticsInfo>
+        <logUrl>http://0.0.0.0:8188/applicationhistory/logs/localhost:9105/container_1430424020775_0001_01_000006/container_1430424020775_0001_01_000006/zshen</logUrl>
+        <containerExitStatus>-105</containerExitStatus>
+        <containerState>COMPLETE</containerState>
+        <nodeHttpAddress>http://localhost:8042</nodeHttpAddress>
+      </container>
+      <container>
+        <containerId>container_1430424020775_0001_01_000005</containerId>
+        <allocatedMB>1024</allocatedMB>
+        <allocatedVCores>1</allocatedVCores>
+        <assignedNodeId>localhost:9105</assignedNodeId>
+        <priority>20</priority>
+        <startedTime>1430424060316</startedTime>
+        <finishedTime>1430424068294</finishedTime>
+        <elapsedTime>7978</elapsedTime>
+        <diagnosticsInfo>Container killed by the ApplicationMaster.
+          Container killed on request. Exit code is 143
+          Container exited with a non-zero exit code 143
+        </diagnosticsInfo>
+        <logUrl>http://0.0.0.0:8188/applicationhistory/logs/localhost:9105/container_1430424020775_0001_01_000005/container_1430424020775_0001_01_000005/zshen</logUrl>
+        <containerExitStatus>-105</containerExitStatus>
+        <containerState>COMPLETE</containerState>
+        <nodeHttpAddress>http://localhost:8042</nodeHttpAddress>
+      </container>
+      <container>
+        <containerId>container_1430424020775_0001_01_000003</containerId>
+        <allocatedMB>1024</allocatedMB>
+        <allocatedVCores>1</allocatedVCores>
+        <assignedNodeId>localhost:9105</assignedNodeId>
+        <priority>20</priority>
+        <startedTime>1430424060315</startedTime>
+        <finishedTime>1430424068289</finishedTime>
+        <elapsedTime>7974</elapsedTime>
+        <diagnosticsInfo>Container killed by the ApplicationMaster.
+          Container killed on request. Exit code is 143
+          Container exited with a non-zero exit code 143
+        </diagnosticsInfo>
+        <logUrl>http://0.0.0.0:8188/applicationhistory/logs/localhost:9105/container_1430424020775_0001_01_000003/container_1430424020775_0001_01_000003/zshen</logUrl>
+        <containerExitStatus>-105</containerExitStatus>
+        <containerState>COMPLETE</containerState>
+        <nodeHttpAddress>http://localhost:8042</nodeHttpAddress>
+      </container>
+      <container>
+        <containerId>container_1430424020775_0001_01_000004</containerId>
+        <allocatedMB>1024</allocatedMB>
+        <allocatedVCores>1</allocatedVCores>
+        <assignedNodeId>localhost:9105</assignedNodeId>
+        <priority>20</priority>
+        <startedTime>1430424060315</startedTime>
+        <finishedTime>1430424068291</finishedTime>
+        <elapsedTime>7976</elapsedTime>
+        <diagnosticsInfo>Container killed by the ApplicationMaster.
+          Container killed on request. Exit code is 143
+          Container exited with a non-zero exit code 143
+        </diagnosticsInfo>
+        <logUrl>http://0.0.0.0:8188/applicationhistory/logs/localhost:9105/container_1430424020775_0001_01_000004/container_1430424020775_0001_01_000004/zshen</logUrl>
+        <containerExitStatus>-105</containerExitStatus>
+        <containerState>COMPLETE</containerState>
+        <nodeHttpAddress>http://localhost:8042</nodeHttpAddress>
+      </container>
+      <container>
+        <containerId>container_1430424020775_0001_01_000002</containerId>
+        <allocatedMB>1024</allocatedMB>
+        <allocatedVCores>1</allocatedVCores>
+        <assignedNodeId>localhost:9105</assignedNodeId>
+        <priority>20</priority>
+        <startedTime>1430424060313</startedTime>
+        <finishedTime>1430424067250</finishedTime>
+        <elapsedTime>6937</elapsedTime>
+        <diagnosticsInfo>Container killed by the ApplicationMaster.
+          Container killed on request. Exit code is 143
+          Container exited with a non-zero exit code 143
+        </diagnosticsInfo>
+        <logUrl>http://0.0.0.0:8188/applicationhistory/logs/localhost:9105/container_1430424020775_0001_01_000002/container_1430424020775_0001_01_000002/zshen</logUrl>
+        <containerExitStatus>-105</containerExitStatus>
+        <containerState>COMPLETE</containerState>
+        <nodeHttpAddress>http://localhost:8042</nodeHttpAddress>
+      </container>
+      <container>
+        <containerId>container_1430424020775_0001_01_000001</containerId>
+        <allocatedMB>2048</allocatedMB>
+        <allocatedVCores>1</allocatedVCores>
+        <assignedNodeId>localhost:9105</assignedNodeId>
+        <priority>0</priority>
+        <startedTime>1430424054314</startedTime>
+        <finishedTime>1430424079022</finishedTime>
+        <elapsedTime>24708</elapsedTime>
+        <diagnosticsInfo></diagnosticsInfo>
+        <logUrl>http://0.0.0.0:8188/applicationhistory/logs/localhost:9105/container_1430424020775_0001_01_000001/container_1430424020775_0001_01_000001/zshen</logUrl>
+        <containerExitStatus>0</containerExitStatus>
+        <containerState>COMPLETE</containerState>
+        <nodeHttpAddress>http://localhost:8042</nodeHttpAddress>
+      </container>
+    </containers>
+
+## <a name="REST_API_CONTAINER"></a>Container
+
+With the Container API, you can get a container resource contains information
+about a particular container of an application attempt of an application that
+was running on an YARN cluster.
+
+### URI:
+
+Use the following URI to obtain a container object identified by the `appid`
+value, the `appattemptid` value and the `containerid` value.
+
+    http(s)://<timeline server http(s) address:port>/ws/v1/applicationhistory/apps/{appid}/appattempts/{appattemptid}/containers/{containerid}
+
+### HTTP Operations Supported:
+
+   GET
+
+### Query Parameters Supported:
+
+None
+
+### Elements of the `container` (Container) Object:
+
+
+| Item | Data Type   | Description   |
+|:---- |:---- |:---- |
+| `containerId` | string  | The container Id |
+| `containerState` | string | The container state according to the ResourceManager - valid values are members of the ContainerState enum: COMPLETE |
+| `containerExitStatus` | int  | The container exit status |
+| `logUrl` | string | The log URL that can be used to access the container aggregated log |
+| `diagnosticsInfo` | string | Detailed diagnostics information |
+| `startedTime` | long | The time in which container started (in ms since epoch) |
+| `finishedTime` | long | The time in which the container finished (in ms since epoch) |
+| `elapsedTime` | long | The elapsed time since the container started (in ms) |
+| `allocatedMB` | int | The memory in MB allocated to the container |
+| `allocatedVCores` | int | The virtual cores allocated to the container |
+| `priority` | int  | The priority of the container |
+| `assignedNodeId` | string | The assigned node host and port of the container |
+
+### Response Examples:
+
+#### JSON response
+
+HTTP Request:
+
+    GET http://localhost:8188/ws/v1/applicationhistory/apps/application_1430424020775_0001/appattempts/appattempt_1430424020775_0001_000001/containers/container_1430424020775_0001_01_000001
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/json
+    Transfer-Encoding: chunked
+
+Response Body:
+
+    {
+      "containerId": "container_1430424020775_0001_01_000001",
+      "allocatedMB": 2048,
+      "allocatedVCores": 1,
+      "assignedNodeId": "localhost:9105",
+      "priority": 0,
+      "startedTime": 1430424054314,
+      "finishedTime": 1430424079022,
+      "elapsedTime": 24708,
+      "diagnosticsInfo": "",
+      "logUrl": "http://0.0.0.0:8188/applicationhistory/logs/localhost:9105/container_1430424020775_0001_01_000001/container_1430424020775_0001_01_000001/zshen",
+      "containerExitStatus": 0,
+      "containerState": "COMPLETE",
+      "nodeHttpAddress": "http://localhost:8042"
+    }
+
+#### XML response
+
+HTTP Request:
+
+    GET http://localhost:8188/ws/v1/applicationhistory/apps/application_1430424020775_0001/appattempts/appattempt_1430424020775_0001_000001/containers/container_1430424020775_0001_01_000001
+    Accept: application/xml
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/xml
+    Content-Length: 669
+
+Response Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <container>
+      <containerId>container_1430424020775_0001_01_000001</containerId>
+      <allocatedMB>2048</allocatedMB>
+      <allocatedVCores>1</allocatedVCores>
+      <assignedNodeId>localhost:9105</assignedNodeId>
+      <priority>0</priority>
+      <startedTime>1430424054314</startedTime>
+      <finishedTime>1430424079022</finishedTime>
+      <elapsedTime>24708</elapsedTime>
+      <diagnosticsInfo></diagnosticsInfo>
+      <logUrl>http://0.0.0.0:8188/applicationhistory/logs/localhost:9105/container_1430424020775_0001_01_000001/container_1430424020775_0001_01_000001/zshen</logUrl>
+      <containerExitStatus>0</containerExitStatus>
+      <containerState>COMPLETE</containerState>
+      <nodeHttpAddress>http://localhost:8042</nodeHttpAddress>
+    </container>
+ 
+ 
+### Response Codes
 
-  * On modification of Primary filter value, new value will be appended with the old value.
+1. Queries where a domain, entity type, entity ID or similar cannot be resolved result in
+HTTP 404, "Not Found" responses.
+1. Requests in which the path, parameters or values are invalid result in
+Bad Request, 400, responses.
+1. In a secure cluster, a 401, "Forbidden", response is generated
+when attempting to perform operations to which the caller does
+not have the sufficient rights. There is an exception to this when
+querying some entities, such as Domains; here the API deliberately
+downgrades permission-denied outcomes as empty and not-founds responses.
+This hides details of other domains from an unauthorized caller.
+1. If the content of timeline entity PUT operations is invalid,
+this failure *will not* result in an HTTP error code being retured.
+A status code of 200 will be returned —however, there will be an error code
+in the list of failed entities for each entity which could not be added.


[07/37] hadoop git commit: MAPREDUCE-6361. NPE issue in shuffle caused by concurrent issue between copySucceeded() in one thread and copyFailed() in another thread on the same host. Contributed by Junping Du.

Posted by ji...@apache.org.
MAPREDUCE-6361. NPE issue in shuffle caused by concurrent issue between copySucceeded() in one thread and copyFailed() in another thread on the same host. 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/f4e2b3cc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f4e2b3cc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f4e2b3cc

Branch: refs/heads/HDFS-7240
Commit: f4e2b3cc0b1f4e49c306bc09a9dddd0495225bb2
Parents: 6d5da94
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Wed May 13 00:28:17 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Wed May 13 00:28:17 2015 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  4 ++
 .../task/reduce/ShuffleSchedulerImpl.java       | 14 +++-
 .../task/reduce/TestShuffleScheduler.java       | 70 ++++++++++++++++++++
 3 files changed, 85 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4e2b3cc/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index ca92a97..15cdf90 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -414,6 +414,10 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6360. TestMapreduceConfigFields is placed in wrong dir, 
     introducing compile error (Arshad Mohammad via vinayakumarb)
 
+    MAPREDUCE-6361. NPE issue in shuffle caused by concurrent issue between
+    copySucceeded() in one thread and copyFailed() in another thread on the
+    same host. (Junping Du via ozawa)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4e2b3cc/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleSchedulerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleSchedulerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleSchedulerImpl.java
index 8317672..ff0bb4f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleSchedulerImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleSchedulerImpl.java
@@ -239,7 +239,7 @@ public class ShuffleSchedulerImpl<K,V> implements ShuffleScheduler<K,V> {
   }
   
   private void updateStatus() {
-    updateStatus(null);	
+    updateStatus(null);
   }
 
   public synchronized void hostFailed(String hostname) {
@@ -263,9 +263,17 @@ public class ShuffleSchedulerImpl<K,V> implements ShuffleScheduler<K,V> {
       failureCounts.put(mapId, new IntWritable(1));
     }
     String hostname = host.getHostName();
+    IntWritable hostFailedNum = hostFailures.get(hostname);
+    // MAPREDUCE-6361: hostname could get cleanup from hostFailures in another
+    // thread with copySucceeded.
+    // In this case, add back hostname to hostFailures to get rid of NPE issue.
+    if (hostFailedNum == null) {
+      hostFailures.put(hostname, new IntWritable(1));
+    }
     //report failure if already retried maxHostFailures times
-    boolean hostFail = hostFailures.get(hostname).get() > getMaxHostFailures() ? true : false;
-    
+    boolean hostFail = hostFailures.get(hostname).get() >
+        getMaxHostFailures() ? true : false;
+
     if (failures >= abortFailureLimit) {
       try {
         throw new IOException(failures + " failures downloading " + mapId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4e2b3cc/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestShuffleScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestShuffleScheduler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestShuffleScheduler.java
index 6ac2320..654b748 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestShuffleScheduler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestShuffleScheduler.java
@@ -213,6 +213,76 @@ public class TestShuffleScheduler {
     Assert.assertEquals(copyMessage(10, 1, 2), progress.toString());
   }
 
+  @SuppressWarnings("rawtypes")
+  @Test
+  public <K, V> void TestSucceedAndFailedCopyMap() throws Exception {
+    JobConf job = new JobConf();
+    job.setNumMapTasks(2);
+    //mock creation
+    TaskUmbilicalProtocol mockUmbilical = mock(TaskUmbilicalProtocol.class);
+    Reporter mockReporter = mock(Reporter.class);
+    FileSystem mockFileSystem = mock(FileSystem.class);
+    Class<? extends org.apache.hadoop.mapred.Reducer>  combinerClass = job.getCombinerClass();
+    @SuppressWarnings("unchecked")  // needed for mock with generic
+    CombineOutputCollector<K, V>  mockCombineOutputCollector =
+        (CombineOutputCollector<K, V>) mock(CombineOutputCollector.class);
+    org.apache.hadoop.mapreduce.TaskAttemptID mockTaskAttemptID =
+        mock(org.apache.hadoop.mapreduce.TaskAttemptID.class);
+    LocalDirAllocator mockLocalDirAllocator = mock(LocalDirAllocator.class);
+    CompressionCodec mockCompressionCodec = mock(CompressionCodec.class);
+    Counter mockCounter = mock(Counter.class);
+    TaskStatus mockTaskStatus = mock(TaskStatus.class);
+    Progress mockProgress = mock(Progress.class);
+    MapOutputFile mockMapOutputFile = mock(MapOutputFile.class);
+    Task mockTask = mock(Task.class);
+    @SuppressWarnings("unchecked")
+    MapOutput<K, V> output = mock(MapOutput.class);
+
+    ShuffleConsumerPlugin.Context<K, V> context =
+        new ShuffleConsumerPlugin.Context<K, V>(
+            mockTaskAttemptID, job, mockFileSystem,
+            mockUmbilical, mockLocalDirAllocator,
+            mockReporter, mockCompressionCodec,
+            combinerClass, mockCombineOutputCollector,
+            mockCounter, mockCounter, mockCounter,
+            mockCounter, mockCounter, mockCounter,
+            mockTaskStatus, mockProgress, mockProgress,
+            mockTask, mockMapOutputFile, null);
+    TaskStatus status = new TaskStatus() {
+      @Override
+      public boolean getIsMap() {
+        return false;
+      }
+      @Override
+      public void addFetchFailedMap(TaskAttemptID mapTaskId) {
+      }
+    };
+    Progress progress = new Progress();
+    ShuffleSchedulerImpl<K, V> scheduler = new ShuffleSchedulerImpl<K, V>(job,
+        status, null, null, progress, context.getShuffledMapsCounter(),
+        context.getReduceShuffleBytes(), context.getFailedShuffleCounter());
+
+    MapHost host1 = new MapHost("host1", null);
+    TaskAttemptID failedAttemptID = new TaskAttemptID(
+        new org.apache.hadoop.mapred.TaskID(
+        new JobID("test",0), TaskType.MAP, 0), 0);
+
+    TaskAttemptID succeedAttemptID = new TaskAttemptID(
+        new org.apache.hadoop.mapred.TaskID(
+        new JobID("test",0), TaskType.MAP, 1), 1);
+
+    // handle output fetch failure for failedAttemptID, part I
+    scheduler.hostFailed(host1.getHostName());
+
+    // handle output fetch succeed for succeedAttemptID
+    long bytes = (long)500 * 1024 * 1024;
+    scheduler.copySucceeded(succeedAttemptID, host1, bytes, 0, 500000, output);
+
+    // handle output fetch failure for failedAttemptID, part II
+    // for MAPREDUCE-6361: verify no NPE exception get thrown out
+    scheduler.copyFailed(failedAttemptID, host1, true, false);
+  }
+
   private static String copyMessage(int attemptNo, double rate1, double rate2) {
     int attemptZero = attemptNo - 1;
     return String.format("copy task(attempt_test_0000_m_%06d_%d succeeded at %1.2f MB/s)"


[12/37] hadoop git commit: HDFS-6184. Capture NN's thread dump when it fails over. Contributed by Ming Ma.

Posted by ji...@apache.org.
HDFS-6184. Capture NN's thread dump when it fails over. Contributed by Ming Ma.


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

Branch: refs/heads/HDFS-7240
Commit: 2463666ecb553dbde1b8c540a21ad3d599239acf
Parents: f24452d
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed May 13 11:37:22 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed May 13 11:37:22 2015 +0900

----------------------------------------------------------------------
 .../apache/hadoop/ha/ZKFailoverController.java  |   5 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   4 +-
 .../hdfs/tools/DFSZKFailoverController.java     |  60 +++++
 .../src/main/resources/hdfs-default.xml         |  11 +
 .../ha/TestDFSZKFailoverController.java         | 226 -----------------
 .../hdfs/tools/TestDFSZKFailoverController.java | 243 +++++++++++++++++++
 7 files changed, 322 insertions(+), 230 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2463666e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
index 9eb1ff8..788d48e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
@@ -844,12 +844,11 @@ public abstract class ZKFailoverController {
    * @return the last health state passed to the FC
    * by the HealthMonitor.
    */
-  @VisibleForTesting
-  synchronized State getLastHealthState() {
+  protected synchronized State getLastHealthState() {
     return lastHealthState;
   }
 
-  private synchronized void setLastHealthState(HealthMonitor.State newState) {
+  protected synchronized void setLastHealthState(HealthMonitor.State newState) {
     LOG.info("Local service " + localTarget +
         " entered state: " + newState);
     lastHealthState = newState;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2463666e/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 cd477af..135b50c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -546,6 +546,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8255. Rename getBlockReplication to getPreferredBlockReplication.
     (Contributed by Zhe Zhang)
 
+    HDFS-6184. Capture NN's thread dump when it fails over.
+    (Ming Ma via aajisaka)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2463666e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 4356b9b..f8e9f3a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -543,7 +543,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final boolean DFS_HA_AUTO_FAILOVER_ENABLED_DEFAULT = false;
   public static final String DFS_HA_ZKFC_PORT_KEY = "dfs.ha.zkfc.port";
   public static final int DFS_HA_ZKFC_PORT_DEFAULT = 8019;
-  
+  public static final String DFS_HA_ZKFC_NN_HTTP_TIMEOUT_KEY = "dfs.ha.zkfc.nn.http.timeout.ms";
+  public static final int DFS_HA_ZKFC_NN_HTTP_TIMEOUT_KEY_DEFAULT = 20000;
+
   // Security-related configs
   public static final String DFS_ENCRYPT_DATA_TRANSFER_KEY = "dfs.encrypt.data.transfer";
   public static final boolean DFS_ENCRYPT_DATA_TRANSFER_DEFAULT = false;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2463666e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
index 4e256a2..f125a27 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
@@ -20,15 +20,20 @@ package org.apache.hadoop.hdfs.tools;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
 
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.net.HttpURLConnection;
 import java.net.InetSocketAddress;
+import java.net.URL;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ha.HAServiceTarget;
+import org.apache.hadoop.ha.HealthMonitor;
 import org.apache.hadoop.ha.ZKFailoverController;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -37,6 +42,7 @@ import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.ha.proto.HAZKInfoProtos.ActiveNodeInfo;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.SecurityUtil;
@@ -57,6 +63,9 @@ public class DFSZKFailoverController extends ZKFailoverController {
   /* the same as superclass's localTarget, but with the more specfic NN type */
   private final NNHAServiceTarget localNNTarget;
 
+  // This is used only for unit tests
+  private boolean isThreadDumpCaptured = false;
+
   @Override
   protected HAServiceTarget dataToTarget(byte[] data) {
     ActiveNodeInfo proto;
@@ -201,4 +210,55 @@ public class DFSZKFailoverController extends ZKFailoverController {
     LOG.warn(msg);
     throw new AccessControlException(msg);
   }
+
+  /**
+   * capture local NN's thread dump and write it to ZKFC's log.
+   */
+  private void getLocalNNThreadDump() {
+    isThreadDumpCaptured = false;
+    // We use the same timeout value for both connection establishment
+    // timeout and read timeout.
+    int httpTimeOut = conf.getInt(
+        DFSConfigKeys.DFS_HA_ZKFC_NN_HTTP_TIMEOUT_KEY,
+        DFSConfigKeys.DFS_HA_ZKFC_NN_HTTP_TIMEOUT_KEY_DEFAULT);
+    if (httpTimeOut == 0) {
+      // If timeout value is set to zero, the feature is turned off.
+      return;
+    }
+    try {
+      String stacksUrl = DFSUtil.getInfoServer(localNNTarget.getAddress(),
+          conf, DFSUtil.getHttpClientScheme(conf)) + "/stacks";
+      URL url = new URL(stacksUrl);
+      HttpURLConnection conn = (HttpURLConnection)url.openConnection();
+      conn.setReadTimeout(httpTimeOut);
+      conn.setConnectTimeout(httpTimeOut);
+      conn.connect();
+      ByteArrayOutputStream out = new ByteArrayOutputStream();
+      IOUtils.copyBytes(conn.getInputStream(), out, 4096, true);
+      StringBuilder localNNThreadDumpContent =
+          new StringBuilder("-- Local NN thread dump -- \n");
+      localNNThreadDumpContent.append(out);
+      localNNThreadDumpContent.append("\n -- Local NN thread dump -- ");
+      LOG.info(localNNThreadDumpContent);
+      isThreadDumpCaptured = true;
+    } catch (IOException e) {
+      LOG.warn("Can't get local NN thread dump due to " + e.getMessage());
+    }
+  }
+
+  @Override
+  protected synchronized void setLastHealthState(HealthMonitor.State newState) {
+    super.setLastHealthState(newState);
+    // Capture local NN thread dump when the target NN health state changes.
+    if (getLastHealthState() == HealthMonitor.State.SERVICE_NOT_RESPONDING ||
+        getLastHealthState() == HealthMonitor.State.SERVICE_UNHEALTHY) {
+      getLocalNNThreadDump();
+    }
+  }
+
+  @VisibleForTesting
+  boolean isThreadDumpCaptured() {
+    return isThreadDumpCaptured;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2463666e/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index fe1d1de..5d1d670 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -2334,4 +2334,15 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.ha.zkfc.nn.http.timeout.ms</name>
+  <value>20000</value>
+  <description>
+    The HTTP connection and read timeout value (unit is ms ) when DFS ZKFC
+    tries to get local NN thread dump after local NN becomes
+    SERVICE_NOT_RESPONDING or SERVICE_UNHEALTHY.
+    If it is set to zero, DFS ZKFC won't get local NN thread dump.
+  </description>
+</property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2463666e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSZKFailoverController.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSZKFailoverController.java
deleted file mode 100644
index bcbd543..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSZKFailoverController.java
+++ /dev/null
@@ -1,226 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.namenode.ha;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.concurrent.TimeoutException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.ha.ClientBaseWithFixes;
-import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
-import org.apache.hadoop.ha.HealthMonitor;
-import org.apache.hadoop.ha.TestNodeFencer.AlwaysSucceedFencer;
-import org.apache.hadoop.ha.ZKFCTestUtil;
-import org.apache.hadoop.ha.ZKFailoverController;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.tools.DFSHAAdmin;
-import org.apache.hadoop.hdfs.tools.DFSZKFailoverController;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
-import org.apache.hadoop.test.MultithreadedTestUtil.TestingThread;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.google.common.base.Supplier;
-
-public class TestDFSZKFailoverController extends ClientBaseWithFixes {
-  private Configuration conf;
-  private MiniDFSCluster cluster;
-  private TestContext ctx;
-  private ZKFCThread thr1, thr2;
-  private FileSystem fs;
-
-  static {
-    // Make tests run faster by avoiding fsync()
-    EditLogFileOutputStream.setShouldSkipFsyncForTesting(true);
-  }
-  
-  @Before
-  public void setup() throws Exception {
-    conf = new Configuration();
-    // Specify the quorum per-nameservice, to ensure that these configs
-    // can be nameservice-scoped.
-    conf.set(ZKFailoverController.ZK_QUORUM_KEY + ".ns1", hostPort);
-    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY,
-        AlwaysSucceedFencer.class.getName());
-    conf.setBoolean(DFSConfigKeys.DFS_HA_AUTO_FAILOVER_ENABLED_KEY, true);
-
-    // Turn off IPC client caching, so that the suite can handle
-    // the restart of the daemons between test cases.
-    conf.setInt(
-        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
-        0);
-    
-    conf.setInt(DFSConfigKeys.DFS_HA_ZKFC_PORT_KEY + ".ns1.nn1", 10023);
-    conf.setInt(DFSConfigKeys.DFS_HA_ZKFC_PORT_KEY + ".ns1.nn2", 10024);
-
-    MiniDFSNNTopology topology = new MiniDFSNNTopology()
-    .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
-        .addNN(new MiniDFSNNTopology.NNConf("nn1").setIpcPort(10021))
-        .addNN(new MiniDFSNNTopology.NNConf("nn2").setIpcPort(10022)));
-    cluster = new MiniDFSCluster.Builder(conf)
-        .nnTopology(topology)
-        .numDataNodes(0)
-        .build();
-    cluster.waitActive();
-
-    ctx = new TestContext();
-    ctx.addThread(thr1 = new ZKFCThread(ctx, 0));
-    assertEquals(0, thr1.zkfc.run(new String[]{"-formatZK"}));
-
-    thr1.start();
-    waitForHAState(0, HAServiceState.ACTIVE);
-    
-    ctx.addThread(thr2 = new ZKFCThread(ctx, 1));
-    thr2.start();
-    
-    // Wait for the ZKFCs to fully start up
-    ZKFCTestUtil.waitForHealthState(thr1.zkfc,
-        HealthMonitor.State.SERVICE_HEALTHY, ctx);
-    ZKFCTestUtil.waitForHealthState(thr2.zkfc,
-        HealthMonitor.State.SERVICE_HEALTHY, ctx);
-    
-    fs = HATestUtil.configureFailoverFs(cluster, conf);
-  }
-  
-  @After
-  public void shutdown() throws Exception {
-    cluster.shutdown();
-    
-    if (thr1 != null) {
-      thr1.interrupt();
-    }
-    if (thr2 != null) {
-      thr2.interrupt();
-    }
-    if (ctx != null) {
-      ctx.stop();
-    }
-  }
-  
-  /**
-   * Test that automatic failover is triggered by shutting the
-   * active NN down.
-   */
-  @Test(timeout=60000)
-  public void testFailoverAndBackOnNNShutdown() throws Exception {
-    Path p1 = new Path("/dir1");
-    Path p2 = new Path("/dir2");
-    
-    // Write some data on the first NN
-    fs.mkdirs(p1);
-    // Shut it down, causing automatic failover
-    cluster.shutdownNameNode(0);
-    // Data should still exist. Write some on the new NN
-    assertTrue(fs.exists(p1));
-    fs.mkdirs(p2);
-    assertEquals(AlwaysSucceedFencer.getLastFencedService().getAddress(),
-        thr1.zkfc.getLocalTarget().getAddress());
-    
-    // Start the first node back up
-    cluster.restartNameNode(0);
-    // This should have no effect -- the new node should be STANDBY.
-    waitForHAState(0, HAServiceState.STANDBY);
-    assertTrue(fs.exists(p1));
-    assertTrue(fs.exists(p2));
-    // Shut down the second node, which should failback to the first
-    cluster.shutdownNameNode(1);
-    waitForHAState(0, HAServiceState.ACTIVE);
-
-    // First node should see what was written on the second node while it was down.
-    assertTrue(fs.exists(p1));
-    assertTrue(fs.exists(p2));
-    assertEquals(AlwaysSucceedFencer.getLastFencedService().getAddress(),
-        thr2.zkfc.getLocalTarget().getAddress());
-  }
-  
-  @Test(timeout=30000)
-  public void testManualFailover() throws Exception {
-    thr2.zkfc.getLocalTarget().getZKFCProxy(conf, 15000).gracefulFailover();
-    waitForHAState(0, HAServiceState.STANDBY);
-    waitForHAState(1, HAServiceState.ACTIVE);
-
-    thr1.zkfc.getLocalTarget().getZKFCProxy(conf, 15000).gracefulFailover();
-    waitForHAState(0, HAServiceState.ACTIVE);
-    waitForHAState(1, HAServiceState.STANDBY);
-  }
-  
-  @Test(timeout=30000)
-  public void testManualFailoverWithDFSHAAdmin() throws Exception {
-    DFSHAAdmin tool = new DFSHAAdmin();
-    tool.setConf(conf);
-    assertEquals(0, 
-        tool.run(new String[]{"-failover", "nn1", "nn2"}));
-    waitForHAState(0, HAServiceState.STANDBY);
-    waitForHAState(1, HAServiceState.ACTIVE);
-    assertEquals(0,
-        tool.run(new String[]{"-failover", "nn2", "nn1"}));
-    waitForHAState(0, HAServiceState.ACTIVE);
-    waitForHAState(1, HAServiceState.STANDBY);
-  }
-  
-  private void waitForHAState(int nnidx, final HAServiceState state)
-      throws TimeoutException, InterruptedException {
-    final NameNode nn = cluster.getNameNode(nnidx);
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        try {
-          return nn.getRpcServer().getServiceStatus().getState() == state;
-        } catch (Exception e) {
-          e.printStackTrace();
-          return false;
-        }
-      }
-    }, 50, 15000);
-  }
-
-  /**
-   * Test-thread which runs a ZK Failover Controller corresponding
-   * to a given NameNode in the minicluster.
-   */
-  private class ZKFCThread extends TestingThread {
-    private final DFSZKFailoverController zkfc;
-
-    public ZKFCThread(TestContext ctx, int idx) {
-      super(ctx);
-      this.zkfc = DFSZKFailoverController.create(
-          cluster.getConfiguration(idx));
-    }
-
-    @Override
-    public void doWork() throws Exception {
-      try {
-        assertEquals(0, zkfc.run(new String[0]));
-      } catch (InterruptedException ie) {
-        // Interrupted by main thread, that's OK.
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2463666e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSZKFailoverController.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSZKFailoverController.java
new file mode 100644
index 0000000..3e1c96f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSZKFailoverController.java
@@ -0,0 +1,243 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.tools;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.ha.ClientBaseWithFixes;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.ha.HealthMonitor;
+import org.apache.hadoop.ha.TestNodeFencer.AlwaysSucceedFencer;
+import org.apache.hadoop.ha.ZKFCTestUtil;
+import org.apache.hadoop.ha.ZKFailoverController;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
+import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeResourceChecker;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
+import org.apache.hadoop.test.MultithreadedTestUtil.TestingThread;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.base.Supplier;
+import org.mockito.Mockito;
+
+public class TestDFSZKFailoverController extends ClientBaseWithFixes {
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+  private TestContext ctx;
+  private ZKFCThread thr1, thr2;
+  private FileSystem fs;
+
+  static {
+    // Make tests run faster by avoiding fsync()
+    EditLogFileOutputStream.setShouldSkipFsyncForTesting(true);
+  }
+  
+  @Before
+  public void setup() throws Exception {
+    conf = new Configuration();
+    // Specify the quorum per-nameservice, to ensure that these configs
+    // can be nameservice-scoped.
+    conf.set(ZKFailoverController.ZK_QUORUM_KEY + ".ns1", hostPort);
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY,
+        AlwaysSucceedFencer.class.getName());
+    conf.setBoolean(DFSConfigKeys.DFS_HA_AUTO_FAILOVER_ENABLED_KEY, true);
+
+    // Turn off IPC client caching, so that the suite can handle
+    // the restart of the daemons between test cases.
+    conf.setInt(
+        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
+        0);
+    
+    conf.setInt(DFSConfigKeys.DFS_HA_ZKFC_PORT_KEY + ".ns1.nn1", 10023);
+    conf.setInt(DFSConfigKeys.DFS_HA_ZKFC_PORT_KEY + ".ns1.nn2", 10024);
+
+    MiniDFSNNTopology topology = new MiniDFSNNTopology()
+    .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
+        .addNN(new MiniDFSNNTopology.NNConf("nn1").setIpcPort(10021))
+        .addNN(new MiniDFSNNTopology.NNConf("nn2").setIpcPort(10022)));
+    cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(topology)
+        .numDataNodes(0)
+        .build();
+    cluster.waitActive();
+
+    ctx = new TestContext();
+    ctx.addThread(thr1 = new ZKFCThread(ctx, 0));
+    assertEquals(0, thr1.zkfc.run(new String[]{"-formatZK"}));
+
+    thr1.start();
+    waitForHAState(0, HAServiceState.ACTIVE);
+    
+    ctx.addThread(thr2 = new ZKFCThread(ctx, 1));
+    thr2.start();
+    
+    // Wait for the ZKFCs to fully start up
+    ZKFCTestUtil.waitForHealthState(thr1.zkfc,
+        HealthMonitor.State.SERVICE_HEALTHY, ctx);
+    ZKFCTestUtil.waitForHealthState(thr2.zkfc,
+        HealthMonitor.State.SERVICE_HEALTHY, ctx);
+    
+    fs = HATestUtil.configureFailoverFs(cluster, conf);
+  }
+  
+  @After
+  public void shutdown() throws Exception {
+    cluster.shutdown();
+    
+    if (thr1 != null) {
+      thr1.interrupt();
+    }
+    if (thr2 != null) {
+      thr2.interrupt();
+    }
+    if (ctx != null) {
+      ctx.stop();
+    }
+  }
+
+  /**
+   * Test that thread dump is captured after NN state changes.
+   */
+  @Test(timeout=60000)
+  public void testThreadDumpCaptureAfterNNStateChange() throws Exception {
+    NameNodeResourceChecker mockResourceChecker = Mockito.mock(
+        NameNodeResourceChecker.class);
+    Mockito.doReturn(false).when(mockResourceChecker).hasAvailableDiskSpace();
+    cluster.getNameNode(0).getNamesystem()
+        .setNNResourceChecker(mockResourceChecker);
+    waitForHAState(0, HAServiceState.STANDBY);
+    while (!thr1.zkfc.isThreadDumpCaptured()) {
+      Thread.sleep(1000);
+    }
+  }
+
+  /**
+   * Test that automatic failover is triggered by shutting the
+   * active NN down.
+   */
+  @Test(timeout=60000)
+  public void testFailoverAndBackOnNNShutdown() throws Exception {
+    Path p1 = new Path("/dir1");
+    Path p2 = new Path("/dir2");
+
+    // Write some data on the first NN
+    fs.mkdirs(p1);
+    // Shut it down, causing automatic failover
+    cluster.shutdownNameNode(0);
+    // Data should still exist. Write some on the new NN
+    assertTrue(fs.exists(p1));
+    fs.mkdirs(p2);
+    assertEquals(AlwaysSucceedFencer.getLastFencedService().getAddress(),
+        thr1.zkfc.getLocalTarget().getAddress());
+    
+    // Start the first node back up
+    cluster.restartNameNode(0);
+    // This should have no effect -- the new node should be STANDBY.
+    waitForHAState(0, HAServiceState.STANDBY);
+    assertTrue(fs.exists(p1));
+    assertTrue(fs.exists(p2));
+    // Shut down the second node, which should failback to the first
+    cluster.shutdownNameNode(1);
+    waitForHAState(0, HAServiceState.ACTIVE);
+
+    // First node should see what was written on the second node while it was down.
+    assertTrue(fs.exists(p1));
+    assertTrue(fs.exists(p2));
+    assertEquals(AlwaysSucceedFencer.getLastFencedService().getAddress(),
+        thr2.zkfc.getLocalTarget().getAddress());
+  }
+  
+  @Test(timeout=30000)
+  public void testManualFailover() throws Exception {
+    thr2.zkfc.getLocalTarget().getZKFCProxy(conf, 15000).gracefulFailover();
+    waitForHAState(0, HAServiceState.STANDBY);
+    waitForHAState(1, HAServiceState.ACTIVE);
+
+    thr1.zkfc.getLocalTarget().getZKFCProxy(conf, 15000).gracefulFailover();
+    waitForHAState(0, HAServiceState.ACTIVE);
+    waitForHAState(1, HAServiceState.STANDBY);
+  }
+  
+  @Test(timeout=30000)
+  public void testManualFailoverWithDFSHAAdmin() throws Exception {
+    DFSHAAdmin tool = new DFSHAAdmin();
+    tool.setConf(conf);
+    assertEquals(0, 
+        tool.run(new String[]{"-failover", "nn1", "nn2"}));
+    waitForHAState(0, HAServiceState.STANDBY);
+    waitForHAState(1, HAServiceState.ACTIVE);
+    assertEquals(0,
+        tool.run(new String[]{"-failover", "nn2", "nn1"}));
+    waitForHAState(0, HAServiceState.ACTIVE);
+    waitForHAState(1, HAServiceState.STANDBY);
+  }
+
+  private void waitForHAState(int nnidx, final HAServiceState state)
+      throws TimeoutException, InterruptedException {
+    final NameNode nn = cluster.getNameNode(nnidx);
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        try {
+          return nn.getRpcServer().getServiceStatus().getState() == state;
+        } catch (Exception e) {
+          e.printStackTrace();
+          return false;
+        }
+      }
+    }, 50, 15000);
+  }
+
+  /**
+   * Test-thread which runs a ZK Failover Controller corresponding
+   * to a given NameNode in the minicluster.
+   */
+  private class ZKFCThread extends TestingThread {
+    private final DFSZKFailoverController zkfc;
+
+    public ZKFCThread(TestContext ctx, int idx) {
+      super(ctx);
+      this.zkfc = DFSZKFailoverController.create(
+          cluster.getConfiguration(idx));
+    }
+
+    @Override
+    public void doWork() throws Exception {
+      try {
+        assertEquals(0, zkfc.run(new String[0]));
+      } catch (InterruptedException ie) {
+        // Interrupted by main thread, that's OK.
+      }
+    }
+  }
+
+}


[08/37] hadoop git commit: YARN-3629. NodeID is always printed as "null" in node manager initialization log. Contributed by nijel.

Posted by ji...@apache.org.
YARN-3629. NodeID is always printed as "null" in node manager
initialization log. Contributed by nijel.


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

Branch: refs/heads/HDFS-7240
Commit: 5c2f05cd9bad9bf9beb0f4ca18f4ae1bc3e84499
Parents: f4e2b3c
Author: Devaraj K <de...@apache.org>
Authored: Tue May 12 22:20:25 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Tue May 12 22:20:25 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java     | 3 ++-
 2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c2f05cd/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index d5fc259..131161f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -394,6 +394,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3602. TestResourceLocalizationService.testPublicResourceInitializesLocalDir
     fails Intermittently due to IOException from cleanup. (zhihai xu via xgong)
 
+    YARN-3629. NodeID is always printed as "null" in node manager initialization log.
+    (nijel via devaraj)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c2f05cd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index b1ab5f1..0eb7ff4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -199,7 +199,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
         + durationToTrackStoppedContainers);
     }
     super.serviceInit(conf);
-    LOG.info("Initialized nodemanager for " + nodeId + ":" +
+    LOG.info("Initialized nodemanager with :" +
         " physical-memory=" + memoryMb + " virtual-memory=" + virtualMemoryMb +
         " virtual-cores=" + virtualCores);
 
@@ -213,6 +213,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
 
     // NodeManager is the last service to start, so NodeId is available.
     this.nodeId = this.context.getNodeId();
+    LOG.info("Node ID assigned is : " + this.nodeId);
     this.httpPort = this.context.getHttpPort();
     this.nodeManagerVersionId = YarnVersionInfo.getVersion();
     try {


[02/37] hadoop git commit: Move YARN-3493 in CHANGES.txt from 2.8 to 2.7.1

Posted by ji...@apache.org.
Move YARN-3493 in CHANGES.txt from 2.8 to 2.7.1


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

Branch: refs/heads/HDFS-7240
Commit: 3d28611cc6850de129b831158c420f9487103213
Parents: d6f6741
Author: Wangda Tan <wa...@apache.org>
Authored: Mon May 11 18:06:54 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Mon May 11 18:06:54 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3d28611c/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index b5cb0a5..2412dce 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -298,9 +298,6 @@ Release 2.8.0 - UNRELEASED
     YARN-3021. YARN's delegation-token handling disallows certain trust setups
     to operate properly over DistCp. (Yongjun Zhang via jianhe)
 
-    YARN-3493. RM fails to come up with error "Failed to load/recover state" 
-    when mem settings are changed. (Jian He via wangda)
-
     YARN-3136. Fixed a synchronization problem of
     AbstractYarnScheduler#getTransferredContainers. (Sunil G via jianhe)
 
@@ -465,6 +462,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3434. Interaction between reservations and userlimit can result in 
     significant ULF violation (tgraves)
 
+    YARN-3493. RM fails to come up with error "Failed to load/recover state"
+    when mem settings are changed. (Jian He via wangda)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES


[30/37] hadoop git commit: HDFS-8243. Files written by TestHostsFiles and TestNameNodeMXBean are causing Release Audit Warnings. (Contributed by Ruth Wisniewski)

Posted by ji...@apache.org.
HDFS-8243. Files written by TestHostsFiles and TestNameNodeMXBean are causing Release Audit Warnings. (Contributed by Ruth Wisniewski)


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

Branch: refs/heads/HDFS-7240
Commit: 54fa9b421797885c1d3f20db215a883c8fca84e1
Parents: 4356e8a
Author: Arpit Agarwal <ar...@apache.org>
Authored: Wed May 13 19:43:53 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Wed May 13 19:43:53 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt             |  3 +++
 .../hadoop/hdfs/server/namenode/TestHostsFiles.java     | 12 +++++++++++-
 .../hadoop/hdfs/server/namenode/TestNameNodeMXBean.java | 12 ++++++++----
 3 files changed, 22 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/54fa9b42/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 932b500..2e51086 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -752,6 +752,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8380. Always call addStoredBlock on blocks which have been shifted
     from one storage to another (cmccabe)
 
+    HDFS-8243. Files written by TestHostsFiles and TestNameNodeMXBean are
+    causing Release Audit Warnings. (Ruth Wisniewski via Arpit Agarwal)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54fa9b42/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java
index 1806d82..a93cc2a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java
@@ -20,9 +20,11 @@ package org.apache.hadoop.hdfs.server.namenode;
 import static org.junit.Assert.assertTrue;
 
 import java.lang.management.ManagementFactory;
+import java.io.File;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileSystem;
@@ -126,7 +128,12 @@ public class TestHostsFiles {
       assertTrue("Live nodes should contain the decommissioned node",
               nodes.contains("Decommissioned"));
     } finally {
-      cluster.shutdown();
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+      if (localFileSys.exists(dir)) {
+        FileUtils.deleteQuietly(new File(dir.toUri().getPath()));
+      }
     }
   }
 
@@ -167,6 +174,9 @@ public class TestHostsFiles {
       if (cluster != null) {
         cluster.shutdown();
       }
+      if (localFileSys.exists(dir)) {
+        FileUtils.deleteQuietly(new File(dir.toUri().getPath()));
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54fa9b42/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
index c649621..681e8a4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import org.apache.commons.io.FileUtils;
 import com.google.common.util.concurrent.Uninterruptibles;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -214,6 +215,8 @@ public class TestNameNodeMXBean {
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1);
     MiniDFSCluster cluster = null;
+    FileSystem localFileSys = null;
+    Path dir = null;
 
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
@@ -226,10 +229,9 @@ public class TestNameNodeMXBean {
         "Hadoop:service=NameNode,name=NameNodeInfo");
 
       // Define include file to generate deadNodes metrics
-      FileSystem localFileSys = FileSystem.getLocal(conf);
+      localFileSys = FileSystem.getLocal(conf);
       Path workingDir = localFileSys.getWorkingDirectory();
-      Path dir = new Path(workingDir,
-        "build/test/data/temp/TestNameNodeMXBean");
+      dir = new Path(workingDir,"build/test/data/temp/TestNameNodeMXBean");
       Path includeFile = new Path(dir, "include");
       assertTrue(localFileSys.mkdirs(dir));
       StringBuilder includeHosts = new StringBuilder();
@@ -258,8 +260,10 @@ public class TestNameNodeMXBean {
         assertTrue(deadNode.containsKey("decommissioned"));
         assertTrue(deadNode.containsKey("xferaddr"));
       }
-
     } finally {
+      if ((localFileSys != null) && localFileSys.exists(dir)) {
+        FileUtils.deleteQuietly(new File(dir.toUri().getPath()));
+      }
       if (cluster != null) {
         cluster.shutdown();
       }


[15/37] hadoop git commit: MAPREDUCE-6366. mapreduce.terasort.final.sync configuration in TeraSort doesn't work. Contributed by Takuya Fukudome.

Posted by ji...@apache.org.
MAPREDUCE-6366. mapreduce.terasort.final.sync configuration in TeraSort doesn't work. Contributed by Takuya Fukudome.


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

Branch: refs/heads/HDFS-7240
Commit: e82067bfe680ce04acc0153693cce3cd385e5567
Parents: fcd0702
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Wed May 13 16:44:37 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Wed May 13 16:44:50 2015 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                              | 3 +++
 .../main/java/org/apache/hadoop/examples/terasort/TeraSort.java   | 1 -
 .../org/apache/hadoop/examples/terasort/TeraSortConfigKeys.java   | 2 +-
 3 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e82067bf/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index fc98376..7fe8483 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -419,6 +419,9 @@ Release 2.8.0 - UNRELEASED
     copySucceeded() in one thread and copyFailed() in another thread on the
     same host. (Junping Du via ozawa)
 
+    MAPREDUCE-6366. mapreduce.terasort.final.sync configuration in TeraSort
+    doesn't work. (Takuya Fukudome via ozawa)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e82067bf/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSort.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSort.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSort.java
index 5d586e6..9beff3e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSort.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSort.java
@@ -328,7 +328,6 @@ public class TeraSort extends Configured implements Tool {
     }
     
     job.getConfiguration().setInt("dfs.replication", getOutputReplication(job));
-    TeraOutputFormat.setFinalSync(job, true);
     int ret = job.waitForCompletion(true) ? 0 : 1;
     LOG.info("done");
     return ret;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e82067bf/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSortConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSortConfigKeys.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSortConfigKeys.java
index 0822a50..0e7a534 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSortConfigKeys.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSortConfigKeys.java
@@ -70,7 +70,7 @@ public enum TeraSortConfigKeys {
   public static final long DEFAULT_NUM_ROWS = 0L;
   public static final int DEFAULT_NUM_PARTITIONS = 10;
   public static final long DEFAULT_SAMPLE_SIZE = 100000L;
-  public static final boolean DEFAULT_FINAL_SYNC_ATTRIBUTE = false;
+  public static final boolean DEFAULT_FINAL_SYNC_ATTRIBUTE = true;
   public static final boolean DEFAULT_USE_TERA_SCHEDULER = true;
   public static final boolean DEFAULT_USE_SIMPLE_PARTITIONER = false;
   public static final int DEFAULT_OUTPUT_REPLICATION = 1;


[28/37] hadoop git commit: YARN-3362. Add node label usage in RM CapacityScheduler web UI. (Naganarasimha G R via wangda)

Posted by ji...@apache.org.
YARN-3362. Add node label usage in RM CapacityScheduler web UI. (Naganarasimha G R via wangda)


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

Branch: refs/heads/HDFS-7240
Commit: 0e85044e26da698c45185585310ae0e99448cd80
Parents: 281d47a
Author: Wangda Tan <wa...@apache.org>
Authored: Wed May 13 17:00:36 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed May 13 17:00:36 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../scheduler/capacity/AbstractCSQueue.java     |   2 +-
 .../webapp/CapacitySchedulerPage.java           | 161 +++++++++++++++----
 .../resourcemanager/webapp/RMWebServices.java   |   6 +-
 .../webapp/dao/CapacitySchedulerInfo.java       |  37 +++--
 .../dao/CapacitySchedulerLeafQueueInfo.java     |   4 +-
 .../webapp/dao/CapacitySchedulerQueueInfo.java  |  29 ++--
 .../capacity/TestCapacityScheduler.java         |   9 +-
 8 files changed, 191 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e85044e/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index af8d26f..0346c54 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -238,6 +238,9 @@ Release 2.8.0 - UNRELEASED
 
     YARN-3521. Support return structured NodeLabel objects in REST API (Sunil G via wangda)
 
+    YARN-3362. Add node label usage in RM CapacityScheduler web UI.
+    (Naganarasimha G R via wangda)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e85044e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
index 8b4637a..cd5bd8d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
@@ -556,7 +556,7 @@ public abstract class AbstractCSQueue implements CSQueue {
         queueUsage, nodePartition, cluster, schedulingMode);
   }
   
-  boolean accessibleToPartition(String nodePartition) {
+  public boolean accessibleToPartition(String nodePartition) {
     // if queue's label is *, it can access any node
     if (accessibleLabels != null
         && accessibleLabels.contains(RMNodeLabelsManager.ANY)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e85044e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
index 4381a34..255150e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
@@ -22,14 +22,18 @@ import static org.apache.hadoop.yarn.util.StringHelper.join;
 
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.nodelabels.RMNodeLabel;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
 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.QueueCapacities;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.UserInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerLeafQueueInfo;
@@ -63,48 +67,92 @@ class CapacitySchedulerPage extends RmView {
   static class CSQInfo {
     CapacitySchedulerInfo csinfo;
     CapacitySchedulerQueueInfo qinfo;
+    String label;
   }
 
   static class LeafQueueInfoBlock extends HtmlBlock {
     final CapacitySchedulerLeafQueueInfo lqinfo;
+    private String nodeLabel;
 
     @Inject LeafQueueInfoBlock(ViewContext ctx, CSQInfo info) {
       super(ctx);
       lqinfo = (CapacitySchedulerLeafQueueInfo) info.qinfo;
+      nodeLabel = info.label;
     }
 
     @Override
     protected void render(Block html) {
+      if (nodeLabel == null) {
+        renderLeafQueueInfoWithoutParition(html);
+      } else {
+        renderLeafQueueInfoWithPartition(html);
+      }
+    }
 
-      ResponseInfo ri = info("\'" + lqinfo.getQueuePath().substring(5) + "\' Queue Status").
-          _("Queue State:", lqinfo.getQueueState()).
-          _("Used Capacity:", percent(lqinfo.getUsedCapacity() / 100)).
-          _("Absolute Used Capacity:", percent(lqinfo.getAbsoluteUsedCapacity() / 100)).
-          _("Absolute Capacity:", percent(lqinfo.getAbsoluteCapacity() / 100)).
-          _("Absolute Max Capacity:", percent(lqinfo.getAbsoluteMaxCapacity() / 100)).
-          _("Used Resources:", lqinfo.getResourcesUsed().toString()).
-          _("Num Schedulable Applications:", Integer.toString(lqinfo.getNumActiveApplications())).
-          _("Num Non-Schedulable Applications:", Integer.toString(lqinfo.getNumPendingApplications())).
-          _("Num Containers:", Integer.toString(lqinfo.getNumContainers())).
-          _("Max Applications:", Integer.toString(lqinfo.getMaxApplications())).
-          _("Max Applications Per User:", Integer.toString(lqinfo.getMaxApplicationsPerUser())).
-          _("Max Application Master Resources:", lqinfo.getAMResourceLimit().toString()).
-          _("Used Application Master Resources:", lqinfo.getUsedAMResource().toString()).
-          _("Max Application Master Resources Per User:", lqinfo.getUserAMResourceLimit().toString()).
-          _("Configured Capacity:", percent(lqinfo.getCapacity() / 100)).
-          _("Configured Max Capacity:", percent(lqinfo.getMaxCapacity() / 100)).
-          _("Configured Minimum User Limit Percent:", Integer.toString(lqinfo.getUserLimit()) + "%").
-          _("Configured User Limit Factor:", StringUtils.format(
-              "%.1f", lqinfo.getUserLimitFactor())).
-          _("Accessible Node Labels:", StringUtils.join(",", lqinfo.getNodeLabels())).
-          _("Ordering Policy: ", lqinfo.getOrderingPolicyInfo()).
-          _("Preemption:", lqinfo.getPreemptionDisabled() ? "disabled" : "enabled");
+    private void renderLeafQueueInfoWithPartition(Block html) {
+      nodeLabel = nodeLabel.length() == 0 ? "<DEFAULT_PARTITION>" : nodeLabel;
+      // first display the queue's label specific details :
+      ResponseInfo ri =
+          info("\'" + lqinfo.getQueuePath().substring(5)
+              + "\' Queue Status for Partition \'" + nodeLabel + "\'");
+      renderQueueCapacityInfo(ri);
+      html._(InfoBlock.class);
+      // clear the info contents so this queue's info doesn't accumulate into
+      // another queue's info
+      ri.clear();
+
+      // second display the queue specific details :
+      ri =
+          info("\'" + lqinfo.getQueuePath().substring(5) + "\' Queue Status")
+              ._("Queue State:", lqinfo.getQueueState());
+      renderCommonLeafQueueInfo(ri);
 
       html._(InfoBlock.class);
+      // clear the info contents so this queue's info doesn't accumulate into
+      // another queue's info
+      ri.clear();
+    }
 
-      // clear the info contents so this queue's info doesn't accumulate into another queue's info
+    private void renderLeafQueueInfoWithoutParition(Block html) {
+      ResponseInfo ri =
+          info("\'" + lqinfo.getQueuePath().substring(5) + "\' Queue Status")
+              ._("Queue State:", lqinfo.getQueueState());
+      renderQueueCapacityInfo(ri);
+      renderCommonLeafQueueInfo(ri);
+      html._(InfoBlock.class);
+      // clear the info contents so this queue's info doesn't accumulate into
+      // another queue's info
       ri.clear();
     }
+
+    private void renderQueueCapacityInfo(ResponseInfo ri) {
+      ri.
+      _("Used Capacity:", percent(lqinfo.getUsedCapacity() / 100)).
+      _("Configured Capacity:", percent(lqinfo.getCapacity() / 100)).
+      _("Configured Max Capacity:", percent(lqinfo.getMaxCapacity() / 100)).
+      _("Absolute Used Capacity:", percent(lqinfo.getAbsoluteUsedCapacity() / 100)).
+      _("Absolute Configured Capacity:", percent(lqinfo.getAbsoluteCapacity() / 100)).
+      _("Absolute Configured Max Capacity:", percent(lqinfo.getAbsoluteMaxCapacity() / 100)).
+      _("Used Resources:", lqinfo.getResourcesUsed().toString());
+    }
+
+    private void renderCommonLeafQueueInfo(ResponseInfo ri) {
+      ri.
+      _("Num Schedulable Applications:", Integer.toString(lqinfo.getNumActiveApplications())).
+      _("Num Non-Schedulable Applications:", Integer.toString(lqinfo.getNumPendingApplications())).
+      _("Num Containers:", Integer.toString(lqinfo.getNumContainers())).
+      _("Max Applications:", Integer.toString(lqinfo.getMaxApplications())).
+      _("Max Applications Per User:", Integer.toString(lqinfo.getMaxApplicationsPerUser())).
+      _("Max Application Master Resources:", lqinfo.getAMResourceLimit().toString()).
+      _("Used Application Master Resources:", lqinfo.getUsedAMResource().toString()).
+      _("Max Application Master Resources Per User:", lqinfo.getUserAMResourceLimit().toString()).
+      _("Configured Minimum User Limit Percent:", Integer.toString(lqinfo.getUserLimit()) + "%").
+      _("Configured User Limit Factor:", StringUtils.format(
+          "%.1f", lqinfo.getUserLimitFactor())).
+      _("Accessible Node Labels:", StringUtils.join(",", lqinfo.getNodeLabels())).
+      _("Ordering Policy: ", lqinfo.getOrderingPolicyInfo()).
+      _("Preemption:", lqinfo.getPreemptionDisabled() ? "disabled" : "enabled");
+    }
   }
 
   static class QueueUsersInfoBlock extends HtmlBlock {
@@ -172,7 +220,7 @@ class CapacitySchedulerPage extends RmView {
               span().$style(join(width(absUsedCap/absMaxCap),
                 ";font-size:1px;left:0%;", absUsedCap > absCap ? Q_OVER : Q_UNDER)).
                 _('.')._().
-              span(".q", info.getQueuePath().substring(5))._().
+              span(".q", "Queue: "+info.getQueuePath().substring(5))._().
             span().$class("qstats").$style(left(Q_STATS_POS)).
               _(join(percent(used), " used"))._();
 
@@ -194,11 +242,15 @@ class CapacitySchedulerPage extends RmView {
     final CapacityScheduler cs;
     final CSQInfo csqinfo;
     private final ResourceManager rm;
+    private List<RMNodeLabel> nodeLabelsInfo;
 
     @Inject QueuesBlock(ResourceManager rm, CSQInfo info) {
       cs = (CapacityScheduler) rm.getResourceScheduler();
       csqinfo = info;
       this.rm = rm;
+      RMNodeLabelsManager nodeLabelManager =
+          rm.getRMContext().getNodeLabelManager();
+      nodeLabelsInfo = nodeLabelManager.pullRMNodeLabelsInfo();
     }
 
     @Override
@@ -268,12 +320,6 @@ class CapacitySchedulerPage extends RmView {
               span().$style(Q_END)._("100% ")._().
               span(".q", "default")._()._();
       } else {
-        CSQueue root = cs.getRootQueue();
-        CapacitySchedulerInfo sinfo = new CapacitySchedulerInfo(root, cs);
-        csqinfo.csinfo = sinfo;
-        csqinfo.qinfo = null;
-
-        float used = sinfo.getUsedCapacity() / 100;
         ul.
           li().$style("margin-bottom: 1em").
             span().$style("font-weight: bold")._("Legend:")._().
@@ -285,8 +331,22 @@ class CapacitySchedulerPage extends RmView {
               _("Used (over capacity)")._().
             span().$class("qlegend ui-corner-all ui-state-default").
               _("Max Capacity")._().
-          _().
-          li().
+          _();
+
+        float used = 0;
+        if (null == nodeLabelsInfo
+            || (nodeLabelsInfo.size() == 1 && nodeLabelsInfo.get(0)
+                .getLabelName().isEmpty())) {
+          CSQueue root = cs.getRootQueue();
+          CapacitySchedulerInfo sinfo =
+              new CapacitySchedulerInfo(root, cs, new RMNodeLabel(
+                  RMNodeLabelsManager.NO_LABEL));
+          csqinfo.csinfo = sinfo;
+          csqinfo.qinfo = null;
+
+          used = sinfo.getUsedCapacity() / 100;
+          //label is not enabled in the cluster or there's only "default" label,
+          ul.li().
             a(_Q).$style(width(Q_MAX_WIDTH)).
               span().$style(join(width(used), ";left:0%;",
                   used > 1 ? Q_OVER : Q_UNDER))._(".")._().
@@ -294,6 +354,41 @@ class CapacitySchedulerPage extends RmView {
             span().$class("qstats").$style(left(Q_STATS_POS)).
               _(join(percent(used), " used"))._().
             _(QueueBlock.class)._();
+        } else {
+          for (RMNodeLabel label : nodeLabelsInfo) {
+            CSQueue root = cs.getRootQueue();
+            CapacitySchedulerInfo sinfo =
+                new CapacitySchedulerInfo(root, cs, label);
+            csqinfo.csinfo = sinfo;
+            csqinfo.qinfo = null;
+            csqinfo.label = label.getLabelName();
+            String nodeLabel =
+                csqinfo.label.length() == 0 ? "<DEFAULT_PARTITION>"
+                    : csqinfo.label;
+            QueueCapacities queueCapacities = root.getQueueCapacities();
+            used = queueCapacities.getUsedCapacity(label.getLabelName());
+            String partitionUiTag =
+                "Partition: " + nodeLabel + " " + label.getResource();
+            ul.li().
+            a(_Q).$style(width(Q_MAX_WIDTH)).
+              span().$style(join(width(used), ";left:0%;",
+                  used > 1 ? Q_OVER : Q_UNDER))._(".")._().
+              span(".q", partitionUiTag)._().
+            span().$class("qstats").$style(left(Q_STATS_POS)).
+              _(join(percent(used), " used"))._();
+
+            //for the queue hierarchy under label
+            UL<Hamlet> underLabel = html.ul("#pq");
+            underLabel.li().
+            a(_Q).$style(width(Q_MAX_WIDTH)).
+              span().$style(join(width(used), ";left:0%;",
+                  used > 1 ? Q_OVER : Q_UNDER))._(".")._().
+              span(".q", "Queue: root")._().
+            span().$class("qstats").$style(left(Q_STATS_POS)).
+              _(join(percent(used), " used"))._().
+            _(QueueBlock.class)._()._();
+          }
+        }
       }
       ul._()._().
       script().$type("text/javascript").

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e85044e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index a0a3123..c39f2b9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -104,11 +104,13 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.nodelabels.RMNodeLabel;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
@@ -254,7 +256,9 @@ public class RMWebServices {
     if (rs instanceof CapacityScheduler) {
       CapacityScheduler cs = (CapacityScheduler) rs;
       CSQueue root = cs.getRootQueue();
-      sinfo = new CapacitySchedulerInfo(root, cs);
+      sinfo =
+          new CapacitySchedulerInfo(root, cs, new RMNodeLabel(
+              RMNodeLabelsManager.NO_LABEL));
     } else if (rs instanceof FairScheduler) {
       FairScheduler fs = (FairScheduler) rs;
       sinfo = new FairSchedulerInfo(fs);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e85044e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
index 9901878..f63d02a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
@@ -24,9 +24,12 @@ import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 import javax.xml.bind.annotation.XmlType;
 
+import org.apache.hadoop.yarn.nodelabels.RMNodeLabel;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractCSQueue;
 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.LeafQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
 
 @XmlRootElement(name = "capacityScheduler")
 @XmlType(name = "capacityScheduler")
@@ -46,16 +49,19 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
   public CapacitySchedulerInfo() {
   } // JAXB needs this
 
-  public CapacitySchedulerInfo(CSQueue parent, CapacityScheduler cs) {
+  public CapacitySchedulerInfo(CSQueue parent, CapacityScheduler cs,
+      RMNodeLabel nodeLabel) {
+    String label = nodeLabel.getLabelName();
+    QueueCapacities parentQueueCapacities = parent.getQueueCapacities();
     this.queueName = parent.getQueueName();
-    this.usedCapacity = parent.getUsedCapacity() * 100;
-    this.capacity = parent.getCapacity() * 100;
-    float max = parent.getMaximumCapacity();
+    this.usedCapacity = parentQueueCapacities.getUsedCapacity(label) * 100;
+    this.capacity = parentQueueCapacities.getCapacity(label) * 100;
+    float max = parentQueueCapacities.getMaximumCapacity(label);
     if (max < EPSILON || max > 1f)
       max = 1f;
     this.maxCapacity = max * 100;
 
-    queues = getQueues(parent);
+    queues = getQueues(parent, nodeLabel);
     health = new CapacitySchedulerHealthInfo(cs);
   }
 
@@ -79,16 +85,27 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
     return this.queues;
   }
 
-  protected CapacitySchedulerQueueInfoList getQueues(CSQueue parent) {
+  protected CapacitySchedulerQueueInfoList getQueues(CSQueue parent,
+      RMNodeLabel nodeLabel) {
     CSQueue parentQueue = parent;
-    CapacitySchedulerQueueInfoList queuesInfo = new CapacitySchedulerQueueInfoList();
+    CapacitySchedulerQueueInfoList queuesInfo =
+        new CapacitySchedulerQueueInfoList();
     for (CSQueue queue : parentQueue.getChildQueues()) {
+      if (nodeLabel.getIsExclusive()
+          && !((AbstractCSQueue) queue).accessibleToPartition(nodeLabel
+              .getLabelName())) {
+        // Skip displaying the hierarchy for the queues for which the exclusive
+        // labels are not accessible
+        continue;
+      }
       CapacitySchedulerQueueInfo info;
       if (queue instanceof LeafQueue) {
-        info = new CapacitySchedulerLeafQueueInfo((LeafQueue)queue);
+        info =
+            new CapacitySchedulerLeafQueueInfo((LeafQueue) queue,
+                nodeLabel.getLabelName());
       } else {
-        info = new CapacitySchedulerQueueInfo(queue);
-        info.queues = getQueues(queue);
+        info = new CapacitySchedulerQueueInfo(queue, nodeLabel.getLabelName());
+        info.queues = getQueues(queue, nodeLabel);
       }
       queuesInfo.addToQueueInfoList(info);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e85044e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
index dc61078..ae8d747 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
@@ -47,8 +47,8 @@ public class CapacitySchedulerLeafQueueInfo extends CapacitySchedulerQueueInfo {
   CapacitySchedulerLeafQueueInfo() {
   };
 
-  CapacitySchedulerLeafQueueInfo(LeafQueue q) {
-    super(q);
+  CapacitySchedulerLeafQueueInfo(LeafQueue q, String nodeLabel) {
+    super(q, nodeLabel);
     numActiveApplications = q.getNumActiveApplications();
     numPendingApplications = q.getNumPendingApplications();
     numContainers = q.getNumContainers();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e85044e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
index 40dddea..81b28fd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
@@ -28,8 +28,10 @@ import javax.xml.bind.annotation.XmlSeeAlso;
 import javax.xml.bind.annotation.XmlTransient;
 
 import org.apache.hadoop.yarn.api.records.QueueState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.PlanQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
 
 @XmlRootElement
 @XmlAccessorType(XmlAccessType.FIELD)
@@ -59,28 +61,33 @@ public class CapacitySchedulerQueueInfo {
   CapacitySchedulerQueueInfo() {
   };
 
-  CapacitySchedulerQueueInfo(CSQueue q) {
+  CapacitySchedulerQueueInfo(CSQueue q, String nodeLabel) {
+    QueueCapacities qCapacities = q.getQueueCapacities();
+    ResourceUsage queueResourceUsage = q.getQueueResourceUsage();
+
     queuePath = q.getQueuePath();
-    capacity = q.getCapacity() * 100;
-    usedCapacity = q.getUsedCapacity() * 100;
+    capacity = qCapacities.getCapacity(nodeLabel) * 100;
+    usedCapacity = qCapacities.getUsedCapacity(nodeLabel) * 100;
 
-    maxCapacity = q.getMaximumCapacity();
+    maxCapacity = qCapacities.getMaximumCapacity(nodeLabel);
     if (maxCapacity < EPSILON || maxCapacity > 1f)
       maxCapacity = 1f;
     maxCapacity *= 100;
 
-    absoluteCapacity = cap(q.getAbsoluteCapacity(), 0f, 1f) * 100;
-    absoluteMaxCapacity = cap(q.getAbsoluteMaximumCapacity(), 0f, 1f) * 100;
-    absoluteUsedCapacity = cap(q.getAbsoluteUsedCapacity(), 0f, 1f) * 100;
+    absoluteCapacity =
+        cap(qCapacities.getAbsoluteCapacity(nodeLabel), 0f, 1f) * 100;
+    absoluteMaxCapacity =
+        cap(qCapacities.getAbsoluteMaximumCapacity(nodeLabel), 0f, 1f) * 100;
+    absoluteUsedCapacity =
+        cap(qCapacities.getAbsoluteUsedCapacity(nodeLabel), 0f, 1f) * 100;
     numApplications = q.getNumApplications();
     queueName = q.getQueueName();
     state = q.getState();
-    resourcesUsed = new ResourceInfo(q.getUsedResources());
-    if(q instanceof PlanQueue &&
-       !((PlanQueue)q).showReservationsAsQueues()) {
+    resourcesUsed = new ResourceInfo(queueResourceUsage.getUsed(nodeLabel));
+    if (q instanceof PlanQueue && !((PlanQueue) q).showReservationsAsQueues()) {
       hideReservationQueues = true;
     }
-    
+
     // add labels
     Set<String> labelSet = q.getAccessibleNodeLabels();
     if (labelSet != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e85044e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
index 1a34e57..d360581 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
@@ -73,6 +73,7 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.nodelabels.RMNodeLabel;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.AdminService;
 import org.apache.hadoop.yarn.server.resourcemanager.Application;
@@ -1660,7 +1661,9 @@ public class TestCapacityScheduler {
     CapacityScheduler cs =
         (CapacityScheduler) resourceManager.getResourceScheduler();
     CSQueue origRootQ = cs.getRootQueue();
-    CapacitySchedulerInfo oldInfo = new CapacitySchedulerInfo(origRootQ, cs);
+    CapacitySchedulerInfo oldInfo =
+        new CapacitySchedulerInfo(origRootQ, cs, new RMNodeLabel(
+            RMNodeLabelsManager.NO_LABEL));
     int origNumAppsA = getNumAppsInQueue("a", origRootQ.getChildQueues());
     int origNumAppsRoot = origRootQ.getNumApplications();
 
@@ -1669,7 +1672,9 @@ public class TestCapacityScheduler {
     CSQueue newRootQ = cs.getRootQueue();
     int newNumAppsA = getNumAppsInQueue("a", newRootQ.getChildQueues());
     int newNumAppsRoot = newRootQ.getNumApplications();
-    CapacitySchedulerInfo newInfo = new CapacitySchedulerInfo(newRootQ, cs);
+    CapacitySchedulerInfo newInfo =
+        new CapacitySchedulerInfo(newRootQ, cs, new RMNodeLabel(
+            RMNodeLabelsManager.NO_LABEL));
     CapacitySchedulerLeafQueueInfo origOldA1 =
         (CapacitySchedulerLeafQueueInfo) getQueueInfo("a1", oldInfo.getQueues());
     CapacitySchedulerLeafQueueInfo origNewA1 =


[26/37] hadoop git commit: YARN-3641. NodeManager: stopRecoveryStore() shouldn't be skipped when exceptions happen in stopping NM's sub-services. Contributed by Junping Du

Posted by ji...@apache.org.
YARN-3641. NodeManager: stopRecoveryStore() shouldn't be skipped when exceptions happen in stopping NM's sub-services. 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/711d77cc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/711d77cc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/711d77cc

Branch: refs/heads/HDFS-7240
Commit: 711d77cc54a64b2c3db70bdacc6bf2245c896a4b
Parents: f7de619
Author: Jason Lowe <jl...@apache.org>
Authored: Wed May 13 21:06:47 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed May 13 21:06:47 2015 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                          |  3 +++
 .../hadoop/yarn/server/nodemanager/NodeManager.java      | 11 ++++++++---
 2 files changed, 11 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/711d77cc/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 146690b..af8d26f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -488,6 +488,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3537. NPE when NodeManager.serviceInit fails and stopRecoveryStore
     invoked (Brahma Reddy Battula via jlowe)
 
+    YARN-3641. NodeManager: stopRecoveryStore() shouldn't be skipped when
+    exceptions happen in stopping NM's sub-services. (Junping Du via jlowe)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/711d77cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index 4ac06d0..03e17c8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -326,9 +326,14 @@ public class NodeManager extends CompositeService
     if (isStopping.getAndSet(true)) {
       return;
     }
-    super.serviceStop();
-    stopRecoveryStore();
-    DefaultMetricsSystem.shutdown();
+    try {
+      super.serviceStop();
+      DefaultMetricsSystem.shutdown();
+    } finally {
+      // YARN-3641: NM's services stop get failed shouldn't block the
+      // release of NMLevelDBStore.
+      stopRecoveryStore();
+    }
   }
 
   public String getName() {


[33/37] hadoop git commit: HADOOP-8174. Remove confusing comment in Path#isAbsolute() (Contributed by Suresh Srinivas)

Posted by ji...@apache.org.
HADOOP-8174. Remove confusing comment in Path#isAbsolute() (Contributed by Suresh Srinivas)


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

Branch: refs/heads/HDFS-7240
Commit: 0daa5ada68db483275aaa7f2ed9a2b5eaf5bb9bd
Parents: b2c85db
Author: Vinayakumar B <vi...@apache.org>
Authored: Thu May 14 14:17:36 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Thu May 14 14:17:36 2015 +0530

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt         |  3 +++
 .../src/main/java/org/apache/hadoop/fs/Path.java        | 12 ++----------
 2 files changed, 5 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0daa5ada/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 5e6b9ea..bf39c94 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -677,6 +677,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11361. Fix a race condition in MetricsSourceAdapter.updateJmxCache.
     (Brahma Reddy Battula via ozawa)
 
+    HADOOP-8174. Remove confusing comment in Path#isAbsolute()
+    (Suresh Srinivas via vinayakumarb)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0daa5ada/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
index caeb7a1..a38a46c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
@@ -31,8 +31,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 
 /** Names a file or directory in a {@link FileSystem}.
- * Path strings use slash as the directory separator.  A path string is
- * absolute if it begins with a slash.
+ * Path strings use slash as the directory separator.
  */
 @Stringable
 @InterfaceAudience.Public
@@ -312,14 +311,7 @@ public class Path implements Comparable {
     return uri.getPath().startsWith(SEPARATOR, start);
    }
   
-  /** True if the path component of this URI is absolute. */
-  /**
-   * There is some ambiguity here. An absolute path is a slash
-   * relative name without a scheme or an authority.
-   * So either this method was incorrectly named or its
-   * implementation is incorrect. This method returns true
-   * even if there is a scheme and authority.
-   */
+  /** True if the path is not a relative path and starts with root. */
   public boolean isAbsolute() {
      return isUriPathAbsolute();
   }


[19/37] hadoop git commit: HDFS-8358. TestTraceAdmin fails. Contributed by Masatake Iwasaki.

Posted by ji...@apache.org.
HDFS-8358. TestTraceAdmin fails. Contributed by Masatake Iwasaki.


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

Branch: refs/heads/HDFS-7240
Commit: f9a46a00d2020c6d6466fbc829ada0521cb78dc0
Parents: cdec12d
Author: Kihwal Lee <ki...@apache.org>
Authored: Wed May 13 14:15:27 2015 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Wed May 13 14:15:27 2015 -0500

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                  | 3 +++
 .../test/java/org/apache/hadoop/tracing/TestTraceAdmin.java  | 8 ++++----
 2 files changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9a46a00/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 4fabf97..f4e40b7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -747,6 +747,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8362. Java Compilation Error in TestHdfsConfigFields.java
     (Arshad Mohammad via vinayakumarb)
 
+    HDFS-8358. TestTraceAdmin fails (Masatake Iwasaki via kihwal)
+
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9a46a00/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java
index 4a102a3..acd0dbb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java
@@ -72,10 +72,10 @@ public class TestTraceAdmin {
       Assert.assertEquals("ret:0, [no span receivers found]" + NEWLINE,
           runTraceCommand(trace, "-list", "-host", getHostPortForNN(cluster)));
       Assert.assertEquals("ret:0, Added trace span receiver 1 with " +
-          "configuration local-file-span-receiver.path = " + tracePath + NEWLINE,
+          "configuration dfs.htrace.local-file-span-receiver.path = " + tracePath + NEWLINE,
           runTraceCommand(trace, "-add", "-host", getHostPortForNN(cluster),
               "-class", "org.apache.htrace.impl.LocalFileSpanReceiver",
-              "-Clocal-file-span-receiver.path=" + tracePath));
+              "-Cdfs.htrace.local-file-span-receiver.path=" + tracePath));
       String list =
           runTraceCommand(trace, "-list", "-host", getHostPortForNN(cluster));
       Assert.assertTrue(list.startsWith("ret:0"));
@@ -86,10 +86,10 @@ public class TestTraceAdmin {
       Assert.assertEquals("ret:0, [no span receivers found]" + NEWLINE,
           runTraceCommand(trace, "-list", "-host", getHostPortForNN(cluster)));
       Assert.assertEquals("ret:0, Added trace span receiver 2 with " +
-          "configuration local-file-span-receiver.path = " + tracePath + NEWLINE,
+          "configuration dfs.htrace.local-file-span-receiver.path = " + tracePath + NEWLINE,
           runTraceCommand(trace, "-add", "-host", getHostPortForNN(cluster),
               "-class", "LocalFileSpanReceiver",
-              "-Clocal-file-span-receiver.path=" + tracePath));
+              "-Cdfs.htrace.local-file-span-receiver.path=" + tracePath));
       Assert.assertEquals("ret:0, Removed trace span receiver 2" + NEWLINE,
           runTraceCommand(trace, "-remove", "2", "-host",
               getHostPortForNN(cluster)));


[29/37] hadoop git commit: HADOOP-11361. Fix a race condition in MetricsSourceAdapter.updateJmxCache. Contributed by Brahma Reddy Battula.

Posted by ji...@apache.org.
HADOOP-11361. Fix a race condition in MetricsSourceAdapter.updateJmxCache. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-7240
Commit: 4356e8a5ef0ac6d11a34704b80ef360a710e623a
Parents: 0e85044
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Thu May 14 10:20:45 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Thu May 14 10:20:45 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt    |  3 +++
 .../hadoop/metrics2/impl/MetricsSourceAdapter.java | 17 +++++++----------
 2 files changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4356e8a5/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 207d144..5e6b9ea 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -674,6 +674,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11962. Sasl message with MD5 challenge text shouldn't be LOG out
     even in debug level. (Junping Du via wheat9)
 
+    HADOOP-11361. Fix a race condition in MetricsSourceAdapter.updateJmxCache.
+    (Brahma Reddy Battula via ozawa)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4356e8a5/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSourceAdapter.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSourceAdapter.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSourceAdapter.java
index cae9c3d..f3ddc91 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSourceAdapter.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSourceAdapter.java
@@ -154,31 +154,28 @@ class MetricsSourceAdapter implements DynamicMBean {
 
   private void updateJmxCache() {
     boolean getAllMetrics = false;
-    synchronized(this) {
+    synchronized (this) {
       if (Time.now() - jmxCacheTS >= jmxCacheTTL) {
         // temporarilly advance the expiry while updating the cache
         jmxCacheTS = Time.now() + jmxCacheTTL;
         if (lastRecs == null) {
           getAllMetrics = true;
         }
-      }
-      else {
+      } else {
         return;
       }
-    }
 
-    if (getAllMetrics) {
-      MetricsCollectorImpl builder = new MetricsCollectorImpl();
-      getMetrics(builder, true);
-    }
+      if (getAllMetrics) {
+        MetricsCollectorImpl builder = new MetricsCollectorImpl();
+        getMetrics(builder, true);
+      }
 
-    synchronized(this) {
       updateAttrCache();
       if (getAllMetrics) {
         updateInfoCache();
       }
       jmxCacheTS = Time.now();
-      lastRecs = null;  // in case regular interval update is not running
+      lastRecs = null; // in case regular interval update is not running
     }
   }
 


[09/37] hadoop git commit: HADOOP-11962. Sasl message with MD5 challenge text shouldn't be LOG out even in debug level. Contributed by Junping Du.

Posted by ji...@apache.org.
HADOOP-11962. Sasl message with MD5 challenge text shouldn't be LOG out even in debug level. 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/2f4b6d11
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2f4b6d11
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2f4b6d11

Branch: refs/heads/HDFS-7240
Commit: 2f4b6d1157f280c8a6e1b2e7217fd2ec16991985
Parents: 5c2f05c
Author: Haohui Mai <wh...@apache.org>
Authored: Tue May 12 10:30:32 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue May 12 10:30:32 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                   | 3 +++
 .../hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java | 3 ---
 .../src/main/java/org/apache/hadoop/security/SaslRpcClient.java   | 3 ---
 .../java/org/apache/hadoop/security/UserGroupInformation.java     | 3 ---
 4 files changed, 3 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f4b6d11/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 47731fb..a15444e 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -668,6 +668,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11947. test-patch should return early from determine-issue when
     run in jenkins mode. (Sean Busbey via aw)
 
+    HADOOP-11962. Sasl message with MD5 challenge text shouldn't be LOG out
+    even in debug level. (Junping Du via wheat9)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f4b6d11/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
index 5f1809a..ac32ac9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
@@ -1488,9 +1488,6 @@ public abstract class Server {
     }
     
     private void doSaslReply(Message message) throws IOException {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Sending sasl message "+message);
-      }
       setupResponse(saslResponse, saslCall,
           RpcStatusProto.SUCCESS, null,
           new RpcResponseWrapper(message), null, null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f4b6d11/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java
index 4a1a397..7d3afa8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java
@@ -385,9 +385,6 @@ public class SaslRpcClient {
       }
       RpcSaslProto saslMessage =
           RpcSaslProto.parseFrom(responseWrapper.getMessageBytes());
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Received SASL message "+saslMessage);
-      }
       // handle sasl negotiation process
       RpcSaslProto.Builder response = null;
       switch (saslMessage.getState()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f4b6d11/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
index 4b0b5f3..be3d60d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
@@ -865,9 +865,6 @@ public class UserGroupInformation {
         .getPrivateCredentials(KerberosTicket.class);
     for (KerberosTicket ticket : tickets) {
       if (SecurityUtil.isOriginalTGT(ticket)) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Found tgt " + ticket);
-        }
         return ticket;
       }
     }


[32/37] hadoop git commit: HDFS-7728. Avoid updating quota usage while loading edits. Contributed by Jing Zhao.

Posted by ji...@apache.org.
HDFS-7728. Avoid updating quota usage while loading edits. Contributed by Jing Zhao.


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

Branch: refs/heads/HDFS-7240
Commit: b2c85db86c9a62b0a03ee87547265077f664970a
Parents: 54fa9b4
Author: Haohui Mai <wh...@apache.org>
Authored: Wed May 13 21:50:35 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Wed May 13 21:50:35 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/namenode/FSDirDeleteOp.java     |  61 ++++-----
 .../hdfs/server/namenode/FSDirRenameOp.java     |  24 ++--
 .../hdfs/server/namenode/FSDirSnapshotOp.java   |   8 +-
 .../hdfs/server/namenode/FSDirectory.java       |  18 +++
 .../hdfs/server/namenode/FSEditLogLoader.java   |   7 +-
 .../hadoop/hdfs/server/namenode/FSImage.java    |   4 +-
 .../hadoop/hdfs/server/namenode/INode.java      | 136 ++++++++++++++++---
 .../hdfs/server/namenode/INodeDirectory.java    |  81 ++++++-----
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  83 ++++++-----
 .../hadoop/hdfs/server/namenode/INodeMap.java   |  11 +-
 .../hdfs/server/namenode/INodeReference.java    | 118 ++++++++--------
 .../hdfs/server/namenode/INodeSymlink.java      |  24 +---
 .../hdfs/server/namenode/QuotaCounts.java       |   6 +-
 .../namenode/snapshot/AbstractINodeDiff.java    |  12 +-
 .../snapshot/AbstractINodeDiffList.java         |  23 ++--
 .../snapshot/DirectorySnapshottableFeature.java |  15 +-
 .../snapshot/DirectoryWithSnapshotFeature.java  | 117 +++++++---------
 .../hdfs/server/namenode/snapshot/FileDiff.java |  20 +--
 .../server/namenode/snapshot/FileDiffList.java  |  23 ++--
 .../snapshot/FileWithSnapshotFeature.java       |  27 ++--
 .../namenode/snapshot/SnapshotManager.java      |  14 +-
 .../server/namenode/TestQuotaByStorageType.java |  21 ++-
 .../snapshot/TestFileWithSnapshotFeature.java   |   8 +-
 .../snapshot/TestRenameWithSnapshots.java       |   1 +
 .../namenode/snapshot/TestSnapshotDeletion.java |  12 +-
 .../namenode/snapshot/TestSnapshotManager.java  |   5 +-
 27 files changed, 465 insertions(+), 417 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/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 2e51086..0e6508b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -755,6 +755,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8243. Files written by TestHostsFiles and TestNameNodeMXBean are
     causing Release Audit Warnings. (Ruth Wisniewski via Arpit Agarwal)
 
+    HDFS-7728. Avoid updating quota usage while loading edits.
+    (Jing Zhao via wheat9)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
index f99e50c..962f4b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
+import org.apache.hadoop.hdfs.server.namenode.INode.ReclaimContext;
 import org.apache.hadoop.util.ChunkedArrayList;
 
 import java.io.IOException;
@@ -39,24 +40,26 @@ class FSDirDeleteOp {
    * @param removedINodes INodes that should be removed from inodeMap
    * @return the number of files that have been removed
    */
-  static long delete(
-      FSDirectory fsd, INodesInPath iip, BlocksMapUpdateInfo collectedBlocks,
-      List<INode> removedINodes, List<Long> removedUCFiles,
-      long mtime) throws IOException {
+  static long delete(FSDirectory fsd, INodesInPath iip,
+      BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes,
+      List<Long> removedUCFiles, long mtime) throws IOException {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* FSDirectory.delete: " + iip.getPath());
     }
-    final long filesRemoved;
+    long filesRemoved = -1;
     fsd.writeLock();
     try {
-      if (!deleteAllowed(iip, iip.getPath()) ) {
-        filesRemoved = -1;
-      } else {
+      if (deleteAllowed(iip, iip.getPath()) ) {
         List<INodeDirectory> snapshottableDirs = new ArrayList<>();
         FSDirSnapshotOp.checkSnapshot(iip.getLastINode(), snapshottableDirs);
-        filesRemoved = unprotectedDelete(fsd, iip, collectedBlocks,
-                                         removedINodes, removedUCFiles, mtime);
+        ReclaimContext context = new ReclaimContext(
+            fsd.getBlockStoragePolicySuite(), collectedBlocks, removedINodes,
+            removedUCFiles);
+        if (unprotectedDelete(fsd, iip, context, mtime)) {
+          filesRemoved = context.quotaDelta().getNsDelta();
+        }
         fsd.getFSNamesystem().removeSnapshottableDirs(snapshottableDirs);
+        fsd.updateCount(iip, context.quotaDelta(), false);
       }
     } finally {
       fsd.writeUnlock();
@@ -128,11 +131,13 @@ class FSDirDeleteOp {
     }
     List<INodeDirectory> snapshottableDirs = new ArrayList<>();
     FSDirSnapshotOp.checkSnapshot(iip.getLastINode(), snapshottableDirs);
-    long filesRemoved = unprotectedDelete(
-        fsd, iip, collectedBlocks, removedINodes, removedUCFiles, mtime);
+    boolean filesRemoved = unprotectedDelete(fsd, iip,
+        new ReclaimContext(fsd.getBlockStoragePolicySuite(),
+            collectedBlocks, removedINodes, removedUCFiles),
+        mtime);
     fsn.removeSnapshottableDirs(snapshottableDirs);
 
-    if (filesRemoved >= 0) {
+    if (filesRemoved) {
       fsn.removeLeasesAndINodes(removedUCFiles, removedINodes, false);
       fsn.removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
     }
@@ -213,21 +218,18 @@ class FSDirDeleteOp {
    * Update the count at each ancestor directory with quota
    * @param fsd the FSDirectory instance
    * @param iip the inodes resolved from the path
-   * @param collectedBlocks blocks collected from the deleted path
-   * @param removedINodes inodes that should be removed from inodeMap
-   * @param removedUCFiles inodes whose leases need to be released
+   * @param reclaimContext used to collect blocks and inodes to be removed
    * @param mtime the time the inode is removed
-   * @return the number of inodes deleted; 0 if no inodes are deleted.
+   * @return true if there are inodes deleted
    */
-  private static long unprotectedDelete(
-      FSDirectory fsd, INodesInPath iip, BlocksMapUpdateInfo collectedBlocks,
-      List<INode> removedINodes, List<Long> removedUCFiles, long mtime) {
+  private static boolean unprotectedDelete(FSDirectory fsd, INodesInPath iip,
+      ReclaimContext reclaimContext, long mtime) {
     assert fsd.hasWriteLock();
 
     // check if target node exists
     INode targetNode = iip.getLastINode();
     if (targetNode == null) {
-      return -1;
+      return false;
     }
 
     // record modification
@@ -237,35 +239,24 @@ class FSDirDeleteOp {
     // Remove the node from the namespace
     long removed = fsd.removeLastINode(iip);
     if (removed == -1) {
-      return -1;
+      return false;
     }
 
     // set the parent's modification time
     final INodeDirectory parent = targetNode.getParent();
     parent.updateModificationTime(mtime, latestSnapshot);
 
-    fsd.updateCountForDelete(targetNode, iip);
-    if (removed == 0) {
-      return 0;
-    }
-
     // collect block and update quota
-    INode.ReclaimContext reclaimContext = new INode.ReclaimContext(
-        fsd.getBlockStoragePolicySuite(), collectedBlocks,
-        removedINodes, removedUCFiles);
     if (!targetNode.isInLatestSnapshot(latestSnapshot)) {
       targetNode.destroyAndCollectBlocks(reclaimContext);
     } else {
-      QuotaCounts counts = targetNode.cleanSubtree(reclaimContext,
-          CURRENT_STATE_ID, latestSnapshot);
-      removed = counts.getNameSpace();
-      fsd.updateCountNoQuotaCheck(iip, iip.length() -1, counts.negation());
+      targetNode.cleanSubtree(reclaimContext, CURRENT_STATE_ID, latestSnapshot);
     }
 
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
           + iip.getPath() + " is removed");
     }
-    return removed;
+    return true;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
index 7675907..b69bb42 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
@@ -302,19 +302,18 @@ class FSDirRenameOp {
    * @param timestamp modification time
    * @param options   Rename options
    */
-  static boolean renameForEditLog(
+  static void renameForEditLog(
       FSDirectory fsd, String src, String dst, long timestamp,
       Options.Rename... options)
       throws IOException {
     BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
     final INodesInPath srcIIP = fsd.getINodesInPath4Write(src, false);
     final INodesInPath dstIIP = fsd.getINodesInPath4Write(dst, false);
-    boolean ret = unprotectedRenameTo(fsd, src, dst, srcIIP, dstIIP, timestamp,
+    unprotectedRenameTo(fsd, src, dst, srcIIP, dstIIP, timestamp,
         collectedBlocks, options);
     if (!collectedBlocks.getToDeleteList().isEmpty()) {
       fsd.getFSNamesystem().removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
     }
-    return ret;
   }
 
   /**
@@ -609,7 +608,7 @@ class FSDirRenameOp {
         this.srcIIP = INodesInPath.replace(srcIIP, srcIIP.length() - 1,
             srcChild);
         // get the counts before rename
-        withCount.getReferredINode().computeQuotaUsage(bsps, oldSrcCounts, true);
+        oldSrcCounts.add(withCount.getReferredINode().computeQuotaUsage(bsps));
       } else if (srcChildIsReference) {
         // srcChild is reference but srcChild is not in latest snapshot
         withCount = (INodeReference.WithCount) srcChild.asReference()
@@ -730,18 +729,16 @@ class FSDirRenameOp {
       Preconditions.checkState(oldDstChild != null);
       List<INode> removedINodes = new ChunkedArrayList<>();
       List<Long> removedUCFiles = new ChunkedArrayList<>();
+      INode.ReclaimContext context = new INode.ReclaimContext(bsps,
+          collectedBlocks, removedINodes, removedUCFiles);
       final boolean filesDeleted;
       if (!oldDstChild.isInLatestSnapshot(dstIIP.getLatestSnapshotId())) {
-        oldDstChild.destroyAndCollectBlocks(
-            new INode.ReclaimContext(bsps, collectedBlocks, removedINodes, removedUCFiles));
+        oldDstChild.destroyAndCollectBlocks(context);
         filesDeleted = true;
       } else {
-        filesDeleted = oldDstChild.cleanSubtree(
-            new INode.ReclaimContext(bsps, collectedBlocks, removedINodes,
-                                     removedUCFiles),
-            Snapshot.CURRENT_STATE_ID,
-            dstIIP.getLatestSnapshotId())
-            .getNameSpace() >= 0;
+        oldDstChild.cleanSubtree(context, Snapshot.CURRENT_STATE_ID,
+            dstIIP.getLatestSnapshotId());
+        filesDeleted = context.quotaDelta().getNsDelta() >= 0;
       }
       fsd.getFSNamesystem().removeLeasesAndINodes(
           removedUCFiles, removedINodes, false);
@@ -752,8 +749,7 @@ class FSDirRenameOp {
       // update the quota usage in src tree
       if (isSrcInSnapshot) {
         // get the counts after rename
-        QuotaCounts newSrcCounts = srcChild.computeQuotaUsage(bsps,
-            new QuotaCounts.Builder().build(), false);
+        QuotaCounts newSrcCounts = srcChild.computeQuotaUsage(bsps, false);
         newSrcCounts.subtract(oldSrcCounts);
         srcParent.addSpaceConsumed(newSrcCounts, false);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
index 833bc30..0b28dcb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
@@ -176,11 +176,13 @@ class FSDirSnapshotOp {
     }
 
     INode.BlocksMapUpdateInfo collectedBlocks = new INode.BlocksMapUpdateInfo();
-    ChunkedArrayList<INode> removedINodes = new ChunkedArrayList<INode>();
+    ChunkedArrayList<INode> removedINodes = new ChunkedArrayList<>();
+    INode.ReclaimContext context = new INode.ReclaimContext(
+        fsd.getBlockStoragePolicySuite(), collectedBlocks, removedINodes, null);
     fsd.writeLock();
     try {
-      snapshotManager.deleteSnapshot(iip, snapshotName, collectedBlocks,
-          removedINodes);
+      snapshotManager.deleteSnapshot(iip, snapshotName, context);
+      fsd.updateCount(iip, context.quotaDelta(), false);
       fsd.removeFromInodeMap(removedINodes);
     } finally {
       fsd.writeUnlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index c981626..1583815 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -76,6 +76,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import static org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
@@ -641,6 +642,23 @@ public class FSDirectory implements Closeable {
     }
   }
 
+  public void updateCount(INodesInPath iip, INode.QuotaDelta quotaDelta,
+      boolean check) throws QuotaExceededException {
+    QuotaCounts counts = quotaDelta.getCountsCopy();
+    updateCount(iip, iip.length() - 1, counts.negation(), check);
+    Map<INode, QuotaCounts> deltaInOtherPaths = quotaDelta.getUpdateMap();
+    for (Map.Entry<INode, QuotaCounts> entry : deltaInOtherPaths.entrySet()) {
+      INodesInPath path = INodesInPath.fromINode(entry.getKey());
+      updateCount(path, path.length() - 1, entry.getValue().negation(), check);
+    }
+    for (Map.Entry<INodeDirectory, QuotaCounts> entry :
+        quotaDelta.getQuotaDirMap().entrySet()) {
+      INodeDirectory quotaDir = entry.getKey();
+      quotaDir.getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(
+          entry.getValue().negation());
+    }
+  }
+
   /**
    * Update the quota usage after deletion. The quota update is only necessary
    * when image/edits have been loaded and the file/dir to be deleted is not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index b7a4870..f75c117 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -733,9 +733,10 @@ public class FSEditLogLoader {
           renameReservedPathsOnUpgrade(deleteSnapshotOp.snapshotRoot,
               logVersion);
       INodesInPath iip = fsDir.getINodesInPath4Write(snapshotRoot);
-      fsNamesys.getSnapshotManager().deleteSnapshot(
-          iip, deleteSnapshotOp.snapshotName,
-          collectedBlocks, removedINodes);
+      fsNamesys.getSnapshotManager().deleteSnapshot(iip,
+          deleteSnapshotOp.snapshotName,
+          new INode.ReclaimContext(fsNamesys.dir.getBlockStoragePolicySuite(),
+              collectedBlocks, removedINodes, null));
       fsNamesys.removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
       collectedBlocks.clear();
       fsNamesys.dir.removeFromInodeMap(removedINodes);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
index 6a54967..45184e7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
@@ -882,8 +882,8 @@ public class FSImage implements Closeable {
             child.asDirectory(), counts);
       } else {
         // file or symlink: count here to reduce recursive calls.
-        child.computeQuotaUsage(bsps, childPolicyId, counts, false,
-            Snapshot.CURRENT_STATE_ID);
+        counts.add(child.computeQuotaUsage(bsps, childPolicyId, false,
+            Snapshot.CURRENT_STATE_ID));
       }
     }
       

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
index 64af76f..cf38fa5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
@@ -21,7 +21,10 @@ import java.io.PrintStream;
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.util.List;
+import java.util.Map;
 
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -222,7 +225,8 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
 
   /** Is this inode in the latest snapshot? */
   public final boolean isInLatestSnapshot(final int latestSnapshotId) {
-    if (latestSnapshotId == Snapshot.CURRENT_STATE_ID || latestSnapshotId == Snapshot.NO_SNAPSHOT_ID) {
+    if (latestSnapshotId == Snapshot.CURRENT_STATE_ID ||
+        latestSnapshotId == Snapshot.NO_SNAPSHOT_ID) {
       return false;
     }
     // if parent is a reference node, parent must be a renamed node. We can 
@@ -397,11 +401,10 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    *        The id of the latest snapshot before the to-be-deleted snapshot.
    *        When deleting a current inode, this parameter captures the latest
    *        snapshot.
-   * @return quota usage delta when deleting a snapshot
    */
-  public abstract QuotaCounts cleanSubtree(
-      ReclaimContext reclaimContext, final int snapshotId, int priorSnapshotId);
-  
+  public abstract void cleanSubtree(ReclaimContext reclaimContext,
+      final int snapshotId, int priorSnapshotId);
+
   /**
    * Destroy self and clear everything! If the INode is a file, this method
    * collects its blocks for further block deletion. If the INode is a
@@ -494,8 +497,8 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps) {
     final byte storagePolicyId = isSymlink() ?
         HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
-    return computeQuotaUsage(bsps, storagePolicyId,
-        new QuotaCounts.Builder().build(), true, Snapshot.CURRENT_STATE_ID);
+    return computeQuotaUsage(bsps, storagePolicyId, true,
+        Snapshot.CURRENT_STATE_ID);
   }
 
   /**
@@ -521,25 +524,23 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    *
    * @param bsps Block storage policy suite to calculate intended storage type usage
    * @param blockStoragePolicyId block storage policy id of the current INode
-   * @param counts The subtree counts for returning.
    * @param useCache Whether to use cached quota usage. Note that 
    *                 {@link WithName} node never uses cache for its subtree.
    * @param lastSnapshotId {@link Snapshot#CURRENT_STATE_ID} indicates the 
    *                       computation is in the current tree. Otherwise the id
    *                       indicates the computation range for a 
    *                       {@link WithName} node.
-   * @return The same objects as the counts parameter.
+   * @return The subtree quota counts.
    */
-  public abstract QuotaCounts computeQuotaUsage(
-    BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
-    QuotaCounts counts, boolean useCache, int lastSnapshotId);
+  public abstract QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
+      byte blockStoragePolicyId, boolean useCache, int lastSnapshotId);
 
-  public final QuotaCounts computeQuotaUsage(
-    BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache) {
+  public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
+      boolean useCache) {
     final byte storagePolicyId = isSymlink() ?
         HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
-    return computeQuotaUsage(bsps, storagePolicyId, counts,
-        useCache, Snapshot.CURRENT_STATE_ID);
+    return computeQuotaUsage(bsps, storagePolicyId, useCache,
+        Snapshot.CURRENT_STATE_ID);
   }
 
   /**
@@ -804,6 +805,90 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   }
 
   /**
+   * Information used to record quota usage delta. This data structure is
+   * usually passed along with an operation like {@link #cleanSubtree}. Note
+   * that after the operation the delta counts should be decremented from the
+   * ancestral directories' quota usage.
+   */
+  public static class QuotaDelta {
+    private final QuotaCounts counts;
+    /**
+     * The main usage of this map is to track the quota delta that should be
+     * applied to another path. This usually happens when we reclaim INodes and
+     * blocks while deleting snapshots, and hit an INodeReference. Because the
+     * quota usage for a renamed+snapshotted file/directory is counted in both
+     * the current and historical parents, any change of its quota usage may
+     * need to be propagated along its parent paths both before and after the
+     * rename.
+     */
+    private final Map<INode, QuotaCounts> updateMap;
+
+    /**
+     * When deleting a snapshot we may need to update the quota for directories
+     * with quota feature. This map is used to capture these directories and
+     * their quota usage updates.
+     */
+    private final Map<INodeDirectory, QuotaCounts> quotaDirMap;
+
+    public QuotaDelta() {
+      counts = new QuotaCounts.Builder().build();
+      updateMap = Maps.newHashMap();
+      quotaDirMap = Maps.newHashMap();
+    }
+
+    public void add(QuotaCounts update) {
+      counts.add(update);
+    }
+
+    public void addUpdatePath(INodeReference inode, QuotaCounts update) {
+      QuotaCounts c = updateMap.get(inode);
+      if (c == null) {
+        c = new QuotaCounts.Builder().build();
+        updateMap.put(inode, c);
+      }
+      c.add(update);
+    }
+
+    public void addQuotaDirUpdate(INodeDirectory dir, QuotaCounts update) {
+      Preconditions.checkState(dir.isQuotaSet());
+      QuotaCounts c = quotaDirMap.get(dir);
+      if (c == null) {
+        quotaDirMap.put(dir, update);
+      } else {
+        c.add(update);
+      }
+    }
+
+    public QuotaCounts getCountsCopy() {
+      final QuotaCounts copy = new QuotaCounts.Builder().build();
+      copy.add(counts);
+      return copy;
+    }
+
+    public void setCounts(QuotaCounts c) {
+      this.counts.setNameSpace(c.getNameSpace());
+      this.counts.setStorageSpace(c.getStorageSpace());
+      this.counts.setTypeSpaces(c.getTypeSpaces());
+    }
+
+    public long getNsDelta() {
+      long nsDelta = counts.getNameSpace();
+      for (Map.Entry<INode, QuotaCounts> entry : updateMap.entrySet()) {
+        nsDelta += entry.getValue().getNameSpace();
+      }
+      return nsDelta;
+    }
+
+    public Map<INode, QuotaCounts> getUpdateMap() {
+      return ImmutableMap.copyOf(updateMap);
+    }
+
+    public Map<INodeDirectory, QuotaCounts> getQuotaDirMap() {
+      return ImmutableMap.copyOf(quotaDirMap);
+    }
+  }
+
+  /**
    * Context object to record blocks and inodes that need to be reclaimed
    */
   public static class ReclaimContext {
@@ -811,6 +896,9 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     protected final BlocksMapUpdateInfo collectedBlocks;
     protected final List<INode> removedINodes;
     protected final List<Long> removedUCFiles;
+    /** Used to collect quota usage delta */
+    private final QuotaDelta quotaDelta;
+
     /**
      * @param bsps
      *          block storage policy suite to calculate intended storage type
@@ -830,6 +918,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
       this.collectedBlocks = collectedBlocks;
       this.removedINodes = removedINodes;
       this.removedUCFiles = removedUCFiles;
+      this.quotaDelta = new QuotaDelta();
     }
 
     public BlockStoragePolicySuite storagePolicySuite() {
@@ -839,6 +928,19 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     public BlocksMapUpdateInfo collectedBlocks() {
       return collectedBlocks;
     }
+
+    public QuotaDelta quotaDelta() {
+      return quotaDelta;
+    }
+
+    /**
+     * make a copy with the same collectedBlocks, removedINodes, and
+     * removedUCFiles but a new quotaDelta.
+     */
+    public ReclaimContext getCopy() {
+      return new ReclaimContext(bsps, collectedBlocks, removedINodes,
+          removedUCFiles);
+    }
   }
 
   /**
@@ -851,7 +953,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     private final List<Block> toDeleteList;
     
     public BlocksMapUpdateInfo() {
-      toDeleteList = new ChunkedArrayList<Block>();
+      toDeleteList = new ChunkedArrayList<>();
     }
     
     /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
index 106c9f8..5c437c8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
@@ -558,7 +558,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
    */
   private void addChild(final INode node, final int insertionPoint) {
     if (children == null) {
-      children = new ArrayList<INode>(DEFAULT_FILES_PER_DIRECTORY);
+      children = new ArrayList<>(DEFAULT_FILES_PER_DIRECTORY);
     }
     node.setParent(this);
     children.add(-insertionPoint - 1, node);
@@ -570,10 +570,10 @@ public class INodeDirectory extends INodeWithAdditionalFields
 
   @Override
   public QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
-      byte blockStoragePolicyId, QuotaCounts counts, boolean useCache,
-      int lastSnapshotId) {
+      byte blockStoragePolicyId, boolean useCache, int lastSnapshotId) {
     final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
 
+    QuotaCounts counts = new QuotaCounts.Builder().build();
     // we are computing the quota usage for a specific snapshot here, i.e., the
     // computation only includes files/directories that exist at the time of the
     // given snapshot
@@ -581,9 +581,10 @@ public class INodeDirectory extends INodeWithAdditionalFields
         && !(useCache && isQuotaSet())) {
       ReadOnlyList<INode> childrenList = getChildrenList(lastSnapshotId);
       for (INode child : childrenList) {
-        final byte childPolicyId = child.getStoragePolicyIDForQuota(blockStoragePolicyId);
-        child.computeQuotaUsage(bsps, childPolicyId, counts, useCache,
-            lastSnapshotId);
+        final byte childPolicyId = child.getStoragePolicyIDForQuota(
+            blockStoragePolicyId);
+        counts.add(child.computeQuotaUsage(bsps, childPolicyId, useCache,
+            lastSnapshotId));
       }
       counts.addNameSpace(1);
       return counts;
@@ -605,9 +606,10 @@ public class INodeDirectory extends INodeWithAdditionalFields
       int lastSnapshotId) {
     if (children != null) {
       for (INode child : children) {
-        final byte childPolicyId = child.getStoragePolicyIDForQuota(blockStoragePolicyId);
-        child.computeQuotaUsage(bsps, childPolicyId, counts, useCache,
-            lastSnapshotId);
+        final byte childPolicyId = child.getStoragePolicyIDForQuota(
+            blockStoragePolicyId);
+        counts.add(child.computeQuotaUsage(bsps, childPolicyId, useCache,
+            lastSnapshotId));
       }
     }
     return computeQuotaUsage4CurrentDirectory(bsps, blockStoragePolicyId,
@@ -621,7 +623,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
     // include the diff list
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     if (sf != null) {
-      sf.computeQuotaUsage4CurrentDirectory(bsps, storagePolicyId, counts);
+      counts.add(sf.computeQuotaUsage4CurrentDirectory(bsps, storagePolicyId));
     }
     return counts;
   }
@@ -710,8 +712,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
   public void undoRename4ScrParent(final INodeReference oldChild,
       final INode newChild) throws QuotaExceededException {
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
-    Preconditions.checkState(sf != null,
-        "Directory does not have snapshot feature");
+    assert sf != null : "Directory does not have snapshot feature";
     sf.getDiffs().removeChild(ListType.DELETED, oldChild);
     sf.getDiffs().replaceChild(ListType.CREATED, oldChild, newChild);
     addChild(newChild, true, Snapshot.CURRENT_STATE_ID);
@@ -726,8 +727,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
       final INode deletedChild,
       int latestSnapshotId) throws QuotaExceededException {
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
-    Preconditions.checkState(sf != null,
-        "Directory does not have snapshot feature");
+    assert sf != null : "Directory does not have snapshot feature";
     boolean removeDeletedChild = sf.getDiffs().removeChild(ListType.DELETED,
         deletedChild);
     int sid = removeDeletedChild ? Snapshot.CURRENT_STATE_ID : latestSnapshotId;
@@ -753,10 +753,9 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
 
   /** Call cleanSubtree(..) recursively down the subtree. */
-  public QuotaCounts cleanSubtreeRecursively(
+  public void cleanSubtreeRecursively(
       ReclaimContext reclaimContext, final int snapshot, int prior,
       final Map<INode, INode> excludedNodes) {
-    QuotaCounts counts = new QuotaCounts.Builder().build();
     // in case of deletion snapshot, since this call happens after we modify
     // the diff list, the snapshot to be deleted has been combined or renamed
     // to its latest previous snapshot. (besides, we also need to consider nodes
@@ -765,19 +764,17 @@ public class INodeDirectory extends INodeWithAdditionalFields
     int s = snapshot != Snapshot.CURRENT_STATE_ID
         && prior != Snapshot.NO_SNAPSHOT_ID ? prior : snapshot;
     for (INode child : getChildrenList(s)) {
-      if (snapshot != Snapshot.CURRENT_STATE_ID && excludedNodes != null
-          && excludedNodes.containsKey(child)) {
-        continue;
-      } else {
-        QuotaCounts childCounts = child.cleanSubtree(reclaimContext, snapshot, prior);
-        counts.add(childCounts);
+      if (snapshot == Snapshot.CURRENT_STATE_ID || excludedNodes == null ||
+          !excludedNodes.containsKey(child)) {
+        child.cleanSubtree(reclaimContext, snapshot, prior);
       }
     }
-    return counts;
   }
 
   @Override
   public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
+    reclaimContext.quotaDelta().add(
+        new QuotaCounts.Builder().nameSpace(1).build());
     final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     if (sf != null) {
       sf.clear(reclaimContext, this);
@@ -793,30 +790,30 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
   
   @Override
-  public QuotaCounts cleanSubtree(
-      ReclaimContext reclaimContext, final int snapshotId, int priorSnapshotId) {
+  public void cleanSubtree(ReclaimContext reclaimContext, final int snapshotId,
+      int priorSnapshotId) {
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     // there is snapshot data
     if (sf != null) {
-      return sf.cleanDirectory(reclaimContext, this, snapshotId,
-                               priorSnapshotId);
-    }
-    // there is no snapshot data
-    if (priorSnapshotId == Snapshot.NO_SNAPSHOT_ID
-        && snapshotId == Snapshot.CURRENT_STATE_ID) {
-      // destroy the whole subtree and collect blocks that should be deleted
-      QuotaCounts counts = new QuotaCounts.Builder().build();
-      this.computeQuotaUsage(reclaimContext.bsps, counts, true);
-      destroyAndCollectBlocks(reclaimContext);
-      return counts; 
+      sf.cleanDirectory(reclaimContext, this, snapshotId, priorSnapshotId);
     } else {
-      // process recursively down the subtree
-      QuotaCounts counts = cleanSubtreeRecursively(
-          reclaimContext, snapshotId, priorSnapshotId, null);
-      if (isQuotaSet()) {
-        getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(counts.negation());
+      // there is no snapshot data
+      if (priorSnapshotId == Snapshot.NO_SNAPSHOT_ID &&
+          snapshotId == Snapshot.CURRENT_STATE_ID) {
+        // destroy the whole subtree and collect blocks that should be deleted
+        destroyAndCollectBlocks(reclaimContext);
+      } else {
+        // make a copy the quota delta
+        QuotaCounts old = reclaimContext.quotaDelta().getCountsCopy();
+        // process recursively down the subtree
+        cleanSubtreeRecursively(reclaimContext, snapshotId, priorSnapshotId,
+            null);
+        QuotaCounts current = reclaimContext.quotaDelta().getCountsCopy();
+        current.subtract(old);
+        if (isQuotaSet()) {
+          reclaimContext.quotaDelta().addQuotaDirUpdate(this, current);
+        }
       }
-      return counts;
     }
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 44f23bb..a6ff6fb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -491,37 +491,52 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   @Override
-  public QuotaCounts cleanSubtree(
-      ReclaimContext reclaimContext, final int snapshot, int priorSnapshotId) {
+  public void cleanSubtree(ReclaimContext reclaimContext,
+      final int snapshot, int priorSnapshotId) {
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if (sf != null) {
-      return sf.cleanFile(reclaimContext, this, snapshot, priorSnapshotId);
-    }
-    QuotaCounts counts = new QuotaCounts.Builder().build();
-
-    if (snapshot == CURRENT_STATE_ID) {
-      if (priorSnapshotId == NO_SNAPSHOT_ID) {
-        // this only happens when deleting the current file and the file is not
-        // in any snapshot
-        computeQuotaUsage(reclaimContext.bsps, counts, false);
-        destroyAndCollectBlocks(reclaimContext);
-      } else {
-        FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
-        // when deleting the current file and the file is in snapshot, we should
-        // clean the 0-sized block if the file is UC
-        if (uc != null) {
-          uc.cleanZeroSizeBlock(this, reclaimContext.collectedBlocks);
-          if (reclaimContext.removedUCFiles != null) {
-            reclaimContext.removedUCFiles.add(getId());
+      // TODO: avoid calling getStoragePolicyID
+      sf.cleanFile(reclaimContext, this, snapshot, priorSnapshotId,
+          getStoragePolicyID());
+    } else {
+      if (snapshot == CURRENT_STATE_ID) {
+        if (priorSnapshotId == NO_SNAPSHOT_ID) {
+          // this only happens when deleting the current file and it is not
+          // in any snapshot
+          destroyAndCollectBlocks(reclaimContext);
+        } else {
+          FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
+          // when deleting the current file and it is in snapshot, we should
+          // clean the 0-sized block if the file is UC
+          if (uc != null) {
+            uc.cleanZeroSizeBlock(this, reclaimContext.collectedBlocks);
+            if (reclaimContext.removedUCFiles != null) {
+              reclaimContext.removedUCFiles.add(getId());
+            }
           }
         }
       }
     }
-    return counts;
   }
 
   @Override
   public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
+    // TODO pass in the storage policy
+    reclaimContext.quotaDelta().add(computeQuotaUsage(reclaimContext.bsps,
+        false));
+    clearFile(reclaimContext);
+    FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
+    if (sf != null) {
+      sf.getDiffs().destroyAndCollectSnapshotBlocks(
+          reclaimContext.collectedBlocks);
+      sf.clearDiffs();
+    }
+    if (isUnderConstruction() && reclaimContext.removedUCFiles != null) {
+      reclaimContext.removedUCFiles.add(getId());
+    }
+  }
+
+  public void clearFile(ReclaimContext reclaimContext) {
     if (blocks != null && reclaimContext.collectedBlocks != null) {
       for (BlockInfoContiguous blk : blocks) {
         reclaimContext.collectedBlocks.addDeleteBlock(blk);
@@ -534,15 +549,6 @@ public class INodeFile extends INodeWithAdditionalFields
     }
     clear();
     reclaimContext.removedINodes.add(this);
-    FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
-    if (sf != null) {
-      sf.getDiffs().destroyAndCollectSnapshotBlocks(
-          reclaimContext.collectedBlocks);
-      sf.clearDiffs();
-    }
-    if (isUnderConstruction() && reclaimContext.removedUCFiles != null) {
-      reclaimContext.removedUCFiles.add(getId());
-    }
   }
 
   @Override
@@ -554,18 +560,11 @@ public class INodeFile extends INodeWithAdditionalFields
   // This is the only place that needs to use the BlockStoragePolicySuite to
   // derive the intended storage type usage for quota by storage type
   @Override
-  public final QuotaCounts computeQuotaUsage(
-      BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
-      QuotaCounts counts, boolean useCache,
-      int lastSnapshotId) {
-    long nsDelta = 1;
-    counts.addNameSpace(nsDelta);
-
-    BlockStoragePolicy bsp = null;
-    if (blockStoragePolicyId != BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
-      bsp = bsps.getPolicy(blockStoragePolicyId);
-    }
+  public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
+      byte blockStoragePolicyId, boolean useCache, int lastSnapshotId) {
+    final QuotaCounts counts = new QuotaCounts.Builder().nameSpace(1).build();
 
+    final BlockStoragePolicy bsp = bsps.getPolicy(blockStoragePolicyId);
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if (sf == null) {
       counts.add(storagespaceConsumed(bsp));
@@ -610,7 +609,7 @@ public class INodeFile extends INodeWithAdditionalFields
       final ContentSummaryComputationContext summary) {
     final ContentCounts counts = summary.getCounts();
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
-    long fileLen = 0;
+    final long fileLen;
     if (sf == null) {
       fileLen = computeFileSize();
       counts.addContent(Content.FILE, 1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
index e1cb869..795f474 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
@@ -37,12 +37,12 @@ public class INodeMap {
   static INodeMap newInstance(INodeDirectory rootDir) {
     // Compute the map capacity by allocating 1% of total memory
     int capacity = LightWeightGSet.computeCapacity(1, "INodeMap");
-    GSet<INode, INodeWithAdditionalFields> map
-        = new LightWeightGSet<INode, INodeWithAdditionalFields>(capacity);
+    GSet<INode, INodeWithAdditionalFields> map =
+        new LightWeightGSet<>(capacity);
     map.put(rootDir);
     return new INodeMap(map);
   }
-  
+
   /** Synchronized by external lock. */
   private final GSet<INode, INodeWithAdditionalFields> map;
   
@@ -103,7 +103,7 @@ public class INodeMap {
       @Override
       public QuotaCounts computeQuotaUsage(
           BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
-          QuotaCounts counts, boolean useCache, int lastSnapshotId) {
+          boolean useCache, int lastSnapshotId) {
         return null;
       }
 
@@ -114,9 +114,8 @@ public class INodeMap {
       }
       
       @Override
-      public QuotaCounts cleanSubtree(
+      public void cleanSubtree(
           ReclaimContext reclaimContext, int snapshotId, int priorSnapshotId) {
-          return null;
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
index 90a2eb1..2fea903 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
@@ -25,13 +25,11 @@ import java.util.List;
 
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 import com.google.common.base.Preconditions;
-import org.mortbay.log.Log;
 
 /**
  * An anonymous reference to an inode.
@@ -302,10 +300,9 @@ public abstract class INodeReference extends INode {
   }
 
   @Override // used by WithCount
-  public QuotaCounts cleanSubtree(
+  public void cleanSubtree(
       ReclaimContext reclaimContext, int snapshot, int prior) {
-    return referred.cleanSubtree(reclaimContext,
-                                 snapshot, prior);
+    referred.cleanSubtree(reclaimContext, snapshot, prior);
   }
 
   @Override // used by WithCount
@@ -322,11 +319,10 @@ public abstract class INodeReference extends INode {
   }
 
   @Override
-  public QuotaCounts computeQuotaUsage(
-    BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
-    QuotaCounts counts, boolean useCache, int lastSnapshotId) {
-    return referred.computeQuotaUsage(bsps, blockStoragePolicyId, counts,
-        useCache, lastSnapshotId);
+  public QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
+      byte blockStoragePolicyId, boolean useCache, int lastSnapshotId) {
+    return referred.computeQuotaUsage(bsps, blockStoragePolicyId, useCache,
+        lastSnapshotId);
   }
 
   @Override
@@ -371,9 +367,9 @@ public abstract class INodeReference extends INode {
   
   /** An anonymous reference with reference count. */
   public static class WithCount extends INodeReference {
-    
-    private final List<WithName> withNameList = new ArrayList<WithName>();
-    
+
+    private final List<WithName> withNameList = new ArrayList<>();
+
     /**
      * Compare snapshot with IDs, where null indicates the current status thus
      * is greater than any non-null snapshot.
@@ -507,10 +503,10 @@ public abstract class INodeReference extends INode {
     @Override
     public final ContentSummaryComputationContext computeContentSummary(
         ContentSummaryComputationContext summary) {
-      //only count storagespace for WithName
-      final QuotaCounts q = new QuotaCounts.Builder().build();
-      computeQuotaUsage(summary.getBlockStoragePolicySuite(),
-          getStoragePolicyID(), q, false, lastSnapshotId);
+      // only count storagespace for WithName
+      final QuotaCounts q = computeQuotaUsage(
+          summary.getBlockStoragePolicySuite(), getStoragePolicyID(), false,
+          lastSnapshotId);
       summary.getCounts().addContent(Content.DISKSPACE, q.getStorageSpace());
       summary.getCounts().addTypeSpaces(q.getTypeSpaces());
       return summary;
@@ -518,12 +514,11 @@ public abstract class INodeReference extends INode {
 
     @Override
     public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
-        byte blockStoragePolicyId, QuotaCounts counts, boolean useCache,
-        int lastSnapshotId) {
-      // if this.lastSnapshotId < lastSnapshotId, the rename of the referred 
-      // node happened before the rename of its ancestor. This should be 
-      // impossible since for WithName node we only count its children at the 
-      // time of the rename. 
+        byte blockStoragePolicyId, boolean useCache, int lastSnapshotId) {
+      // if this.lastSnapshotId < lastSnapshotId, the rename of the referred
+      // node happened before the rename of its ancestor. This should be
+      // impossible since for WithName node we only count its children at the
+      // time of the rename.
       Preconditions.checkState(lastSnapshotId == Snapshot.CURRENT_STATE_ID
           || this.lastSnapshotId >= lastSnapshotId);
       final INode referred = this.getReferredINode().asReference()
@@ -534,13 +529,12 @@ public abstract class INodeReference extends INode {
       // been updated by changes in the current tree.
       int id = lastSnapshotId != Snapshot.CURRENT_STATE_ID ? 
           lastSnapshotId : this.lastSnapshotId;
-      return referred.computeQuotaUsage(bsps, blockStoragePolicyId, counts,
-          false, id);
+      return referred.computeQuotaUsage(bsps, blockStoragePolicyId, false, id);
     }
     
     @Override
-    public QuotaCounts cleanSubtree(
-        ReclaimContext reclaimContext, final int snapshot, int prior) {
+    public void cleanSubtree(ReclaimContext reclaimContext, final int snapshot,
+        int prior) {
       // since WithName node resides in deleted list acting as a snapshot copy,
       // the parameter snapshot must be non-null
       Preconditions.checkArgument(snapshot != Snapshot.CURRENT_STATE_ID);
@@ -552,18 +546,18 @@ public abstract class INodeReference extends INode {
       
       if (prior != Snapshot.NO_SNAPSHOT_ID
           && Snapshot.ID_INTEGER_COMPARATOR.compare(snapshot, prior) <= 0) {
-        return new QuotaCounts.Builder().build();
+        return;
       }
 
-      QuotaCounts counts = getReferredINode().cleanSubtree(reclaimContext,
-          snapshot, prior);
+      // record the old quota delta
+      QuotaCounts old = reclaimContext.quotaDelta().getCountsCopy();
+      getReferredINode().cleanSubtree(reclaimContext, snapshot, prior);
       INodeReference ref = getReferredINode().getParentReference();
       if (ref != null) {
-        try {
-          ref.addSpaceConsumed(counts.negation(), true);
-        } catch (QuotaExceededException e) {
-          Log.warn("Should not have QuotaExceededException");
-        }
+        QuotaCounts current = reclaimContext.quotaDelta().getCountsCopy();
+        current.subtract(old);
+        // we need to update the quota usage along the parent path from ref
+        reclaimContext.quotaDelta().addUpdatePath(ref, current);
       }
       
       if (snapshot < lastSnapshotId) {
@@ -571,20 +565,20 @@ public abstract class INodeReference extends INode {
         // in all the nodes existing at the time of the corresponding rename op.
         // Thus if we are deleting a snapshot before/at the snapshot associated 
         // with lastSnapshotId, we do not need to update the quota upwards.
-        counts = new QuotaCounts.Builder().build();
+        reclaimContext.quotaDelta().setCounts(old);
       }
-      return counts;
     }
     
     @Override
     public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
       int snapshot = getSelfSnapshot();
+      reclaimContext.quotaDelta().add(computeQuotaUsage(reclaimContext.bsps));
       if (removeReference(this) <= 0) {
-        getReferredINode().destroyAndCollectBlocks(reclaimContext);
+        getReferredINode().destroyAndCollectBlocks(reclaimContext.getCopy());
       } else {
         int prior = getPriorSnapshot(this);
         INode referred = getReferredINode().asReference().getReferredINode();
-        
+
         if (snapshot != Snapshot.NO_SNAPSHOT_ID) {
           if (prior != Snapshot.NO_SNAPSHOT_ID && snapshot <= prior) {
             // the snapshot to be deleted has been deleted while traversing 
@@ -598,15 +592,13 @@ public abstract class INodeReference extends INode {
             // 5. delete snapshot s2
             return;
           }
-          try {
-            QuotaCounts counts = referred.cleanSubtree(reclaimContext,
-                snapshot, prior);
-            INodeReference ref = getReferredINode().getParentReference();
-            if (ref != null) {
-              ref.addSpaceConsumed(counts.negation(), true);
-            }
-          } catch (QuotaExceededException e) {
-            LOG.error("should not exceed quota while snapshot deletion", e);
+          ReclaimContext newCtx = reclaimContext.getCopy();
+          referred.cleanSubtree(newCtx, snapshot, prior);
+          INodeReference ref = getReferredINode().getParentReference();
+          if (ref != null) {
+            // we need to update the quota usage along the parent path from ref
+            reclaimContext.quotaDelta().addUpdatePath(ref,
+                newCtx.quotaDelta().getCountsCopy());
           }
         }
       }
@@ -654,14 +646,11 @@ public abstract class INodeReference extends INode {
     }
     
     @Override
-    public QuotaCounts cleanSubtree(
-        ReclaimContext reclaimContext, int snapshot, int prior) {
+    public void cleanSubtree(ReclaimContext reclaimContext, int snapshot,
+        int prior) {
       if (snapshot == Snapshot.CURRENT_STATE_ID
           && prior == Snapshot.NO_SNAPSHOT_ID) {
-        QuotaCounts counts = new QuotaCounts.Builder().build();
-        this.computeQuotaUsage(reclaimContext.bsps, counts, true);
         destroyAndCollectBlocks(reclaimContext);
-        return counts;
       } else {
         // if prior is NO_SNAPSHOT_ID, we need to check snapshot belonging to 
         // the previous WithName instance
@@ -674,9 +663,9 @@ public abstract class INodeReference extends INode {
         if (snapshot != Snapshot.CURRENT_STATE_ID
             && prior != Snapshot.NO_SNAPSHOT_ID
             && Snapshot.ID_INTEGER_COMPARATOR.compare(snapshot, prior) <= 0) {
-          return new QuotaCounts.Builder().build();
+          return;
         }
-        return getReferredINode().cleanSubtree(reclaimContext, snapshot, prior);
+        getReferredINode().cleanSubtree(reclaimContext, snapshot, prior);
       }
     }
     
@@ -693,8 +682,15 @@ public abstract class INodeReference extends INode {
      */
     @Override
     public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
+      // since we count everything of the subtree for the quota usage of a
+      // dst reference node, here we should just simply do a quota computation.
+      // then to avoid double counting, we pass a different QuotaDelta to other
+      // calls
+      reclaimContext.quotaDelta().add(computeQuotaUsage(reclaimContext.bsps));
+      ReclaimContext newCtx = reclaimContext.getCopy();
+
       if (removeReference(this) <= 0) {
-        getReferredINode().destroyAndCollectBlocks(reclaimContext);
+        getReferredINode().destroyAndCollectBlocks(newCtx);
       } else {
         // we will clean everything, including files, directories, and 
         // snapshots, that were created after this prior snapshot
@@ -716,18 +712,14 @@ public abstract class INodeReference extends INode {
           // when calling cleanSubtree of the referred node, since we
           // compute quota usage updates before calling this destroy
           // function, we use true for countDiffChange
-          referred.cleanSubtree(reclaimContext, snapshot, prior);
+          referred.cleanSubtree(newCtx, snapshot, prior);
         } else if (referred.isDirectory()) {
           // similarly, if referred is a directory, it must be an
           // INodeDirectory with snapshot
           INodeDirectory dir = referred.asDirectory();
           Preconditions.checkState(dir.isWithSnapshot());
-          try {
-            DirectoryWithSnapshotFeature.destroyDstSubtree(
-                reclaimContext, dir, snapshot, prior);
-          } catch (QuotaExceededException e) {
-            LOG.error("should not exceed quota while snapshot deletion", e);
-          }
+          DirectoryWithSnapshotFeature.destroyDstSubtree(newCtx, dir,
+              snapshot, prior);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
index ad696b4..8ad3aa8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
@@ -72,26 +72,25 @@ public class INodeSymlink extends INodeWithAdditionalFields {
   }
   
   @Override
-  public QuotaCounts cleanSubtree(
-      ReclaimContext reclaimContext, final int snapshotId, int priorSnapshotId) {
+  public void cleanSubtree(ReclaimContext reclaimContext, final int snapshotId,
+      int priorSnapshotId) {
     if (snapshotId == Snapshot.CURRENT_STATE_ID
         && priorSnapshotId == Snapshot.NO_SNAPSHOT_ID) {
       destroyAndCollectBlocks(reclaimContext);
     }
-    return new QuotaCounts.Builder().nameSpace(1).build();
   }
   
   @Override
   public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
     reclaimContext.removedINodes.add(this);
+    reclaimContext.quotaDelta().add(
+        new QuotaCounts.Builder().nameSpace(1).build());
   }
 
   @Override
-  public QuotaCounts computeQuotaUsage(
-      BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
-      QuotaCounts counts, boolean useCache, int lastSnapshotId) {
-    counts.addNameSpace(1);
-    return counts;
+  public QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
+      byte blockStoragePolicyId, boolean useCache, int lastSnapshotId) {
+    return new QuotaCounts.Builder().nameSpace(1).build();
   }
 
   @Override
@@ -108,15 +107,6 @@ public class INodeSymlink extends INodeWithAdditionalFields {
     out.println();
   }
 
-  /**
-   * getAclFeature is not overridden because it is needed for resolving
-   * symlinks.
-  @Override
-  final AclFeature getAclFeature(int snapshotId) {
-    throw new UnsupportedOperationException("ACLs are not supported on symlinks");
-  }
-  */
-
   @Override
   public void removeAclFeature() {
     throw new UnsupportedOperationException("ACLs are not supported on symlinks");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/QuotaCounts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/QuotaCounts.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/QuotaCounts.java
index a3b0448..ef3c2f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/QuotaCounts.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/QuotaCounts.java
@@ -78,14 +78,16 @@ public class QuotaCounts {
     this.tsCounts = builder.tsCounts;
   }
 
-  public void add(QuotaCounts that) {
+  public QuotaCounts add(QuotaCounts that) {
     this.nsSsCounts.add(that.nsSsCounts);
     this.tsCounts.add(that.tsCounts);
+    return this;
   }
 
-  public void subtract(QuotaCounts that) {
+  public QuotaCounts subtract(QuotaCounts that) {
     this.nsSsCounts.subtract(that.nsSsCounts);
     this.tsCounts.subtract(that.tsCounts);
+    return this;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
index 89d3625..2e8620b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
@@ -19,13 +19,9 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.List;
 
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
-import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
 
 import com.google.common.base.Preconditions;
@@ -115,7 +111,7 @@ abstract class AbstractINodeDiff<N extends INode,
   }
 
   /** Combine the posterior diff and collect blocks for deletion. */
-  abstract QuotaCounts combinePosteriorAndCollectBlocks(
+  abstract void combinePosteriorAndCollectBlocks(
       INode.ReclaimContext reclaimContext, final N currentINode,
       final D posterior);
   
@@ -123,11 +119,9 @@ abstract class AbstractINodeDiff<N extends INode,
    * Delete and clear self.
    * @param reclaimContext blocks and inodes that need to be reclaimed
    * @param currentINode The inode where the deletion happens.
-   *
-   * @return usage delta
    */
-  abstract QuotaCounts destroyDiffAndCollectBlocks(
-      INode.ReclaimContext reclaimContext, final N currentINode);
+  abstract void destroyDiffAndCollectBlocks(INode.ReclaimContext reclaimContext,
+      final N currentINode);
 
   @Override
   public String toString() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
index 43127fb..64825f1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
@@ -24,7 +24,6 @@ import java.util.List;
 
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
-import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
 
 /**
  * A list of snapshot diffs for storing snapshot data.
@@ -63,23 +62,20 @@ abstract class AbstractINodeDiffList<N extends INode,
    * @param reclaimContext blocks and inodes that need to be reclaimed
    * @param snapshot The id of the snapshot to be deleted
    * @param prior The id of the snapshot taken before the to-be-deleted snapshot
-   * @return delta in namespace.
+   * @param currentINode the inode where the snapshot diff is deleted
    */
-  public final QuotaCounts deleteSnapshotDiff(
-      INode.ReclaimContext reclaimContext, final int snapshot, final int prior,
-      final N currentINode) {
+  public final void deleteSnapshotDiff(INode.ReclaimContext reclaimContext,
+      final int snapshot, final int prior, final N currentINode) {
     int snapshotIndex = Collections.binarySearch(diffs, snapshot);
-    
-    QuotaCounts counts = new QuotaCounts.Builder().build();
-    D removed = null;
+
+    D removed;
     if (snapshotIndex == 0) {
       if (prior != Snapshot.NO_SNAPSHOT_ID) { // there is still snapshot before
         // set the snapshot to latestBefore
         diffs.get(snapshotIndex).setSnapshotId(prior);
       } else { // there is no snapshot before
         removed = diffs.remove(0);
-        counts.add(removed.destroyDiffAndCollectBlocks(reclaimContext,
-            currentINode));
+        removed.destroyDiffAndCollectBlocks(reclaimContext, currentINode);
       }
     } else if (snapshotIndex > 0) {
       final AbstractINodeDiff<N, A, D> previous = diffs.get(snapshotIndex - 1);
@@ -92,13 +88,12 @@ abstract class AbstractINodeDiffList<N extends INode,
           previous.snapshotINode = removed.snapshotINode;
         }
 
-        counts.add(previous.combinePosteriorAndCollectBlocks(reclaimContext,
-            currentINode, removed));
+        previous.combinePosteriorAndCollectBlocks(reclaimContext, currentINode,
+            removed);
         previous.setPosterior(removed.getPosterior());
         removed.setPosterior(null);
       }
     }
-    return counts;
   }
 
   /** Add an {@link AbstractINodeDiff} for the given snapshot. */
@@ -107,7 +102,7 @@ abstract class AbstractINodeDiffList<N extends INode,
   }
 
   /** Append the diff at the end of the list. */
-  private final D addLast(D diff) {
+  private D addLast(D diff) {
     final D last = getLast();
     diffs.add(diff);
     if (last != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
index 504a64a..4d6ca27 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
@@ -33,14 +33,12 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.Content;
 import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
 import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.SnapshotAndINode;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
-import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.util.Time;
@@ -214,18 +212,7 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
     } else {
       final Snapshot snapshot = snapshotsByNames.get(i);
       int prior = Snapshot.findLatestSnapshot(snapshotRoot, snapshot.getId());
-      try {
-        QuotaCounts counts = snapshotRoot.cleanSubtree(reclaimContext,
-            snapshot.getId(), prior);
-        INodeDirectory parent = snapshotRoot.getParent();
-        if (parent != null) {
-          // there will not be any WithName node corresponding to the deleted
-          // snapshot, thus only update the quota usage in the current tree
-          parent.addSpaceConsumed(counts.negation(), true);
-        }
-      } catch(QuotaExceededException e) {
-        INode.LOG.error("BUG: removeSnapshot increases namespace usage.", e);
-      }
+      snapshotRoot.cleanSubtree(reclaimContext, snapshot.getId(), prior);
       // remove from snapshotsByNames after successfully cleaning the subtree
       snapshotsByNames.remove(i);
       return snapshot;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
index adb012a..e0b4218 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
@@ -49,6 +49,8 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
 import com.google.common.base.Preconditions;
 
+import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID;
+
 /**
  * Feature used to store and process the snapshot diff information for a
  * directory. In particular, it contains a directory diff list recording changes
@@ -95,30 +97,24 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     }
 
     /** clear the created list */
-    private QuotaCounts destroyCreatedList(
-        INode.ReclaimContext reclaimContext, final INodeDirectory currentINode) {
-      QuotaCounts counts = new QuotaCounts.Builder().build();
+    private void destroyCreatedList(INode.ReclaimContext reclaimContext,
+        final INodeDirectory currentINode) {
       final List<INode> createdList = getList(ListType.CREATED);
       for (INode c : createdList) {
-        c.computeQuotaUsage(reclaimContext.storagePolicySuite(), counts, true);
         c.destroyAndCollectBlocks(reclaimContext);
         // c should be contained in the children list, remove it
         currentINode.removeChild(c);
       }
       createdList.clear();
-      return counts;
     }
 
     /** clear the deleted list */
-    private QuotaCounts destroyDeletedList(INode.ReclaimContext reclaimContext) {
-      QuotaCounts counts = new QuotaCounts.Builder().build();
+    private void destroyDeletedList(INode.ReclaimContext reclaimContext) {
       final List<INode> deletedList = getList(ListType.DELETED);
       for (INode d : deletedList) {
-        d.computeQuotaUsage(reclaimContext.storagePolicySuite(), counts, false);
         d.destroyAndCollectBlocks(reclaimContext);
       }
       deletedList.clear();
-      return counts;
     }
 
     /** Serialize {@link #created} */
@@ -202,22 +198,19 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     }
 
     @Override
-    QuotaCounts combinePosteriorAndCollectBlocks(
+    void combinePosteriorAndCollectBlocks(
         final INode.ReclaimContext reclaimContext,
         final INodeDirectory currentDir,
         final DirectoryDiff posterior) {
-      final QuotaCounts counts = new QuotaCounts.Builder().build();
       diff.combinePosterior(posterior.diff, new Diff.Processor<INode>() {
         /** Collect blocks for deleted files. */
         @Override
         public void process(INode inode) {
           if (inode != null) {
-            inode.computeQuotaUsage(reclaimContext.storagePolicySuite(), counts, false);
             inode.destroyAndCollectBlocks(reclaimContext);
           }
         }
       });
-      return counts;
     }
 
     /**
@@ -311,16 +304,14 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     }
 
     @Override
-    QuotaCounts destroyDiffAndCollectBlocks(
+    void destroyDiffAndCollectBlocks(
         INode.ReclaimContext reclaimContext, INodeDirectory currentINode) {
       // this diff has been deleted
-      QuotaCounts counts = new QuotaCounts.Builder().build();
-      counts.add(diff.destroyDeletedList(reclaimContext));
+      diff.destroyDeletedList(reclaimContext);
       INodeDirectoryAttributes snapshotINode = getSnapshotINode();
       if (snapshotINode != null && snapshotINode.getAclFeature() != null) {
         AclStorage.removeAclFeature(snapshotINode.getAclFeature());
       }
-      return counts;
     }
   }
 
@@ -381,7 +372,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
           return diffList.get(i).getSnapshotId();
         }
       }
-      return Snapshot.NO_SNAPSHOT_ID;
+      return NO_SNAPSHOT_ID;
     }
   }
   
@@ -389,7 +380,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     if (diffList == null || diffList.size() == 0) {
       return null;
     }
-    Map<INode, INode> map = new HashMap<INode, INode>(diffList.size());
+    Map<INode, INode> map = new HashMap<>(diffList.size());
     for (INode node : diffList) {
       map.put(node, node);
     }
@@ -399,22 +390,19 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
   /**
    * Destroy a subtree under a DstReference node.
    */
-  public static void destroyDstSubtree(
-      INode.ReclaimContext reclaimContext, INode inode, final int snapshot,
-      final int prior) throws QuotaExceededException {
-    Preconditions.checkArgument(prior != Snapshot.NO_SNAPSHOT_ID);
+  public static void destroyDstSubtree(INode.ReclaimContext reclaimContext,
+      INode inode, final int snapshot, final int prior) {
+    Preconditions.checkArgument(prior != NO_SNAPSHOT_ID);
     if (inode.isReference()) {
       if (inode instanceof INodeReference.WithName
           && snapshot != Snapshot.CURRENT_STATE_ID) {
         // this inode has been renamed before the deletion of the DstReference
         // subtree
-        inode.cleanSubtree(reclaimContext,
-            snapshot, prior);
-      } else { 
+        inode.cleanSubtree(reclaimContext, snapshot, prior);
+      } else {
         // for DstReference node, continue this process to its subtree
         destroyDstSubtree(reclaimContext,
-                          inode.asReference().getReferredINode(), snapshot,
-                          prior);
+            inode.asReference().getReferredINode(), snapshot, prior);
       }
     } else if (inode.isFile()) {
       inode.cleanSubtree(reclaimContext, snapshot, prior);
@@ -455,13 +443,10 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
    * @param inode The inode to clean.
    * @param post The post snapshot.
    * @param prior The id of the prior snapshot.
-   * @return Quota usage update.
    */
-  private static QuotaCounts cleanDeletedINode(
-      INode.ReclaimContext reclaimContext, INode inode, final int post,
-      final int prior) {
-    QuotaCounts counts = new QuotaCounts.Builder().build();
-    Deque<INode> queue = new ArrayDeque<INode>();
+  private static void cleanDeletedINode(INode.ReclaimContext reclaimContext,
+      INode inode, final int post, final int prior) {
+    Deque<INode> queue = new ArrayDeque<>();
     queue.addLast(inode);
     while (!queue.isEmpty()) {
       INode topNode = queue.pollFirst();
@@ -474,7 +459,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
         // prior, we should treat it as regular file/dir
       } else if (topNode.isFile() && topNode.asFile().isWithSnapshot()) {
         INodeFile file = topNode.asFile();
-        counts.add(file.getDiffs().deleteSnapshotDiff(reclaimContext, post, prior, file));
+        file.getDiffs().deleteSnapshotDiff(reclaimContext, post, prior, file);
       } else if (topNode.isDirectory()) {
         INodeDirectory dir = topNode.asDirectory();
         ChildrenDiff priorChildrenDiff = null;
@@ -485,22 +470,19 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
           DirectoryDiff priorDiff = sf.getDiffs().getDiffById(prior);
           if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
             priorChildrenDiff = priorDiff.getChildrenDiff();
-            counts.add(priorChildrenDiff.destroyCreatedList(reclaimContext,
-                dir));
+            priorChildrenDiff.destroyCreatedList(reclaimContext, dir);
           }
         }
-        
+
         for (INode child : dir.getChildrenList(prior)) {
-          if (priorChildrenDiff != null
-              && priorChildrenDiff.search(ListType.DELETED,
-                  child.getLocalNameBytes()) != null) {
+          if (priorChildrenDiff != null && priorChildrenDiff.search(
+              ListType.DELETED, child.getLocalNameBytes()) != null) {
             continue;
           }
           queue.addLast(child);
         }
       }
     }
-    return counts;
   }
 
   /** Diff list sorted by snapshot IDs, i.e. in chronological order. */
@@ -626,13 +608,14 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
   }
 
   public QuotaCounts computeQuotaUsage4CurrentDirectory(
-      BlockStoragePolicySuite bsps, byte storagePolicyId,
-      QuotaCounts counts) {
+      BlockStoragePolicySuite bsps, byte storagePolicyId) {
+    final QuotaCounts counts = new QuotaCounts.Builder().build();
     for(DirectoryDiff d : diffs) {
       for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
-        final byte childPolicyId = deleted.getStoragePolicyIDForQuota(storagePolicyId);
-        deleted.computeQuotaUsage(bsps, childPolicyId, counts, false,
-            Snapshot.CURRENT_STATE_ID);
+        final byte childPolicyId = deleted.getStoragePolicyIDForQuota(
+            storagePolicyId);
+        counts.add(deleted.computeQuotaUsage(bsps, childPolicyId, false,
+            Snapshot.CURRENT_STATE_ID));
       }
     }
     return counts;
@@ -706,28 +689,26 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
     }
   }
 
-  public QuotaCounts cleanDirectory(
-      INode.ReclaimContext reclaimContext, final INodeDirectory currentINode,
-      final int snapshot, int prior) {
-    QuotaCounts counts = new QuotaCounts.Builder().build();
+  public void cleanDirectory(INode.ReclaimContext reclaimContext,
+      final INodeDirectory currentINode, final int snapshot, int prior) {
     Map<INode, INode> priorCreated = null;
     Map<INode, INode> priorDeleted = null;
+    QuotaCounts old = reclaimContext.quotaDelta().getCountsCopy();
     if (snapshot == Snapshot.CURRENT_STATE_ID) { // delete the current directory
       currentINode.recordModification(prior);
       // delete everything in created list
       DirectoryDiff lastDiff = diffs.getLast();
       if (lastDiff != null) {
-        counts.add(lastDiff.diff.destroyCreatedList(reclaimContext,
-            currentINode));
+        lastDiff.diff.destroyCreatedList(reclaimContext, currentINode);
       }
-      counts.add(currentINode.cleanSubtreeRecursively(reclaimContext,
-          snapshot, prior, priorDeleted));
+      currentINode.cleanSubtreeRecursively(reclaimContext, snapshot, prior,
+          null);
     } else {
       // update prior
       prior = getDiffs().updatePrior(snapshot, prior);
       // if there is a snapshot diff associated with prior, we need to record
       // its original created and deleted list before deleting post
-      if (prior != Snapshot.NO_SNAPSHOT_ID) {
+      if (prior != NO_SNAPSHOT_ID) {
         DirectoryDiff priorDiff = this.getDiffs().getDiffById(prior);
         if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
           List<INode> cList = priorDiff.diff.getList(ListType.CREATED);
@@ -736,14 +717,14 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
           priorDeleted = cloneDiffList(dList);
         }
       }
-      
-      counts.add(getDiffs().deleteSnapshotDiff(reclaimContext, snapshot, prior,
-          currentINode));
-      counts.add(currentINode.cleanSubtreeRecursively(reclaimContext,
-          snapshot, prior, priorDeleted));
+
+      getDiffs().deleteSnapshotDiff(reclaimContext, snapshot, prior,
+          currentINode);
+      currentINode.cleanSubtreeRecursively(reclaimContext, snapshot, prior,
+          priorDeleted);
 
       // check priorDiff again since it may be created during the diff deletion
-      if (prior != Snapshot.NO_SNAPSHOT_ID) {
+      if (prior != NO_SNAPSHOT_ID) {
         DirectoryDiff priorDiff = this.getDiffs().getDiffById(prior);
         if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
           // For files/directories created between "prior" and "snapshot", 
@@ -756,8 +737,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
             for (INode cNode : priorDiff.getChildrenDiff().getList(
                 ListType.CREATED)) {
               if (priorCreated.containsKey(cNode)) {
-                counts.add(cNode.cleanSubtree(reclaimContext,
-                    snapshot, Snapshot.NO_SNAPSHOT_ID));
+                cNode.cleanSubtree(reclaimContext, snapshot, NO_SNAPSHOT_ID);
               }
             }
           }
@@ -773,18 +753,17 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
           for (INode dNode : priorDiff.getChildrenDiff().getList(
               ListType.DELETED)) {
             if (priorDeleted == null || !priorDeleted.containsKey(dNode)) {
-              counts.add(cleanDeletedINode(reclaimContext,
-                  dNode, snapshot, prior));
+              cleanDeletedINode(reclaimContext, dNode, snapshot, prior);
             }
           }
         }
       }
     }
 
+    QuotaCounts current = reclaimContext.quotaDelta().getCountsCopy();
+    current.subtract(old);
     if (currentINode.isQuotaSet()) {
-      currentINode.getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(
-          counts.negation());
+      reclaimContext.quotaDelta().addQuotaDirUpdate(currentINode, current);
     }
-    return counts;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
index dd8be82..48c5f33 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
-import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
 
 /**
@@ -79,13 +78,12 @@ public class FileDiff extends
   }
 
   @Override
-  QuotaCounts combinePosteriorAndCollectBlocks(
+  void combinePosteriorAndCollectBlocks(
       INode.ReclaimContext reclaimContext, INodeFile currentINode,
       FileDiff posterior) {
     FileWithSnapshotFeature sf = currentINode.getFileWithSnapshotFeature();
     assert sf != null : "FileWithSnapshotFeature is null";
-    return sf.updateQuotaAndCollectBlocks(reclaimContext,
-        currentINode, posterior);
+    sf.updateQuotaAndCollectBlocks(reclaimContext, currentINode, posterior);
   }
   
   @Override
@@ -109,18 +107,20 @@ public class FileDiff extends
   }
 
   @Override
-  QuotaCounts destroyDiffAndCollectBlocks(
-      INode.ReclaimContext reclaimContext, INodeFile currentINode) {
-    return currentINode.getFileWithSnapshotFeature()
-        .updateQuotaAndCollectBlocks(reclaimContext, currentINode, this);
+  void destroyDiffAndCollectBlocks(INode.ReclaimContext reclaimContext,
+      INodeFile currentINode) {
+    currentINode.getFileWithSnapshotFeature().updateQuotaAndCollectBlocks(
+        reclaimContext, currentINode, this);
   }
 
   public void destroyAndCollectSnapshotBlocks(
       BlocksMapUpdateInfo collectedBlocks) {
-    if(blocks == null || collectedBlocks == null)
+    if (blocks == null || collectedBlocks == null) {
       return;
-    for(BlockInfoContiguous blk : blocks)
+    }
+    for (BlockInfoContiguous blk : blocks) {
       collectedBlocks.addDeleteBlock(blk);
+    }
     blocks = null;
   }
 }


[05/37] hadoop git commit: YARN-3513. Remove unused variables in ContainersMonitorImpl and add debug log for overall resource usage by all containers. Contributed by Naganarasimha G R.

Posted by ji...@apache.org.
YARN-3513. Remove unused variables in ContainersMonitorImpl and add debug
log for overall resource usage by all containers.  Contributed by
Naganarasimha G R.


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

Branch: refs/heads/HDFS-7240
Commit: 8badd82ce256e4dc8c234961120d62a88358ab39
Parents: 360dff5
Author: Devaraj K <de...@apache.org>
Authored: Tue May 12 16:54:38 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Tue May 12 16:54:38 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +++
 .../monitor/ContainersMonitorImpl.java          | 27 ++++++++++++++------
 2 files changed, 22 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8badd82c/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 2412dce..d5fc259 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -227,6 +227,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3587. Fix the javadoc of DelegationTokenSecretManager in yarn, etc. 
     projects. (Gabor Liptak via junping_du)
 
+    YARN-3513. Remove unused variables in ContainersMonitorImpl and add debug
+    log for overall resource usage by all containers. (Naganarasimha G R via devaraj)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8badd82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
index 76bbda1..d1e5e01 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
@@ -389,8 +389,10 @@ public class ContainersMonitorImpl extends AbstractService implements
 
         // Now do the monitoring for the trackingContainers
         // Check memory usage and kill any overflowing containers
-        long vmemStillInUsage = 0;
-        long pmemStillInUsage = 0;
+        long vmemUsageByAllContainers = 0;
+        long pmemByAllContainers = 0;
+        long cpuUsagePercentPerCoreByAllContainers = 0;
+        long cpuUsageTotalCoresByAllContainers = 0;
         for (Iterator<Map.Entry<ContainerId, ProcessTreeInfo>> it =
             trackingContainers.entrySet().iterator(); it.hasNext();) {
 
@@ -504,6 +506,13 @@ public class ContainersMonitorImpl extends AbstractService implements
               containerExitStatus = ContainerExitStatus.KILLED_EXCEEDED_PMEM;
             }
 
+            // Accounting the total memory in usage for all containers
+            vmemUsageByAllContainers += currentVmemUsage;
+            pmemByAllContainers += currentPmemUsage;
+            // Accounting the total cpu usage for all containers
+            cpuUsagePercentPerCoreByAllContainers += cpuUsagePercentPerCore;
+            cpuUsageTotalCoresByAllContainers += cpuUsagePercentPerCore;
+
             if (isMemoryOverLimit) {
               // Virtual or physical memory over limit. Fail the container and
               // remove
@@ -520,12 +529,6 @@ public class ContainersMonitorImpl extends AbstractService implements
                       containerExitStatus, msg));
               it.remove();
               LOG.info("Removed ProcessTree with root " + pId);
-            } else {
-              // Accounting the total memory in usage for all containers that
-              // are still
-              // alive and within limits.
-              vmemStillInUsage += currentVmemUsage;
-              pmemStillInUsage += currentPmemUsage;
             }
           } catch (Exception e) {
             // Log the exception and proceed to the next container.
@@ -533,6 +536,14 @@ public class ContainersMonitorImpl extends AbstractService implements
                 + "while managing memory of " + containerId, e);
           }
         }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Total Resource Usage stats in NM by all containers : "
+              + "Virtual Memory= " + vmemUsageByAllContainers
+              + ", Physical Memory= " + pmemByAllContainers
+              + ", Total CPU usage= " + cpuUsageTotalCoresByAllContainers
+              + ", Total CPU(% per core) usage"
+              + cpuUsagePercentPerCoreByAllContainers);
+        }
 
         try {
           Thread.sleep(monitoringInterval);


[27/37] hadoop git commit: HDFS-8380. Always call addStoredBlock on blocks which have been shifted from one storage to another (cmccabe)

Posted by ji...@apache.org.
HDFS-8380. Always call addStoredBlock on blocks which have been shifted from one storage to another (cmccabe)


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

Branch: refs/heads/HDFS-7240
Commit: 281d47a96937bc329b1b4051ffcb8f5fcac98354
Parents: 711d77c
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed May 13 14:29:05 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Wed May 13 14:29:05 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../blockmanagement/BlockInfoContiguous.java    |  18 ---
 .../server/blockmanagement/BlockManager.java    |  12 +-
 .../TestNameNodePrunesMissingStorages.java      | 118 ++++++++++++++++++-
 4 files changed, 129 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/281d47a9/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 f4e40b7..932b500 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -749,6 +749,8 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8358. TestTraceAdmin fails (Masatake Iwasaki via kihwal)
 
+    HDFS-8380. Always call addStoredBlock on blocks which have been shifted
+    from one storage to another (cmccabe)
 
 Release 2.7.1 - UNRELEASED
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/281d47a9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
index 1ba3536..769046b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
@@ -231,24 +231,6 @@ public class BlockInfoContiguous extends Block
   }
 
   /**
-   * Find specified DatanodeDescriptor.
-   * @return index or -1 if not found.
-   */
-  boolean findDatanode(DatanodeDescriptor dn) {
-    int len = getCapacity();
-    for(int idx = 0; idx < len; idx++) {
-      DatanodeDescriptor cur = getDatanode(idx);
-      if(cur == dn) {
-        return true;
-      }
-      if(cur == null) {
-        break;
-      }
-    }
-    return false;
-  }
-
-  /**
    * Find specified DatanodeStorageInfo.
    * @return DatanodeStorageInfo or null if not found.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/281d47a9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index ab2607b..6d5808e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -1813,6 +1813,9 @@ public class BlockManager {
       if (storageInfo.getBlockReportCount() == 0) {
         // The first block report can be processed a lot more efficiently than
         // ordinary block reports.  This shortens restart times.
+        LOG.info("Processing first storage report for " +
+            storageInfo.getStorageID() + " from datanode " +
+            nodeID.getDatanodeUuid());
         processFirstBlockReport(storageInfo, newReport);
       } else {
         invalidatedBlocks = processReport(storageInfo, newReport);
@@ -2068,7 +2071,12 @@ public class BlockManager {
 
     for (BlockReportReplica iblk : report) {
       ReplicaState reportedState = iblk.getState();
-      
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Initial report of block " + iblk.getBlockName()
+            + " on " + storageInfo.getDatanodeDescriptor() + " size " +
+            iblk.getNumBytes() + " replicaState = " + reportedState);
+      }
       if (shouldPostponeBlocksFromFuture &&
           namesystem.isGenStampInFuture(iblk)) {
         queueReportedBlock(storageInfo, iblk, reportedState,
@@ -2456,7 +2464,7 @@ public class BlockManager {
         storageInfo, ucBlock.reportedBlock, ucBlock.reportedState);
 
     if (ucBlock.reportedState == ReplicaState.FINALIZED &&
-        !block.findDatanode(storageInfo.getDatanodeDescriptor())) {
+        (block.findStorageInfo(storageInfo) < 0)) {
       addStoredBlock(block, storageInfo, null, true);
     }
   } 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/281d47a9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
index 78c1d27..d73f63e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
@@ -23,28 +23,43 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi.FsVolumeReferences;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.io.BufferedOutputStream;
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileOutputStream;
+import java.io.FileReader;
 import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.util.Arrays;
+import java.util.EnumSet;
 import java.util.Iterator;
+import java.util.List;
+import java.util.UUID;
 
 import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.assertThat;
@@ -199,7 +214,7 @@ public class TestNameNodePrunesMissingStorages {
       }
       // Find the volume within the datanode which holds that first storage.
       String volumeDirectoryToRemove = null;
-      try (FsDatasetSpi.FsVolumeReferences volumes =
+      try (FsVolumeReferences volumes =
           datanodeToRemoveStorageFrom.getFSDataset().getFsVolumeReferences()) {
         assertEquals(NUM_STORAGES_PER_DN, volumes.size());
         for (FsVolumeSpi volume : volumes) {
@@ -249,4 +264,105 @@ public class TestNameNodePrunesMissingStorages {
       }
     }
   }
+
+  private static void rewriteVersionFile(File versionFile,
+                            String newStorageId) throws IOException {
+    BufferedReader in = new BufferedReader(new FileReader(versionFile));
+    File newVersionFile =
+        new File(versionFile.getParent(), UUID.randomUUID().toString());
+    Writer out = new BufferedWriter(new OutputStreamWriter(
+        new FileOutputStream(newVersionFile), "UTF-8"));
+    final String STORAGE_ID = "storageID=";
+    boolean success = false;
+    try {
+      String line;
+      while ((line = in.readLine()) != null) {
+        if (line.startsWith(STORAGE_ID)) {
+          out.write(STORAGE_ID + newStorageId + "\n");
+        } else {
+          out.write(line + "\n");
+        }
+      }
+      in.close();
+      in = null;
+      out.close();
+      out = null;
+      newVersionFile.renameTo(versionFile);
+      success = true;
+    } finally {
+      if (in != null) {
+        in.close();
+      }
+      if (out != null) {
+        out.close();
+      }
+      if (!success) {
+        versionFile.delete();
+      }
+    }
+  }
+
+  @Test(timeout=300000)
+  public void testRenamingStorageIds() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 0);
+    final MiniDFSCluster cluster = new MiniDFSCluster
+        .Builder(conf).numDataNodes(1)
+        .storagesPerDatanode(1)
+        .build();
+    GenericTestUtils.setLogLevel(BlockManager.LOG, Level.ALL);
+    try {
+      cluster.waitActive();
+      final Path TEST_PATH = new Path("/foo1");
+      DistributedFileSystem fs = cluster.getFileSystem();
+      // Create a file and leave it open
+      DFSTestUtil.createFile(fs, TEST_PATH, 1, (short)1, 0xdeadbeef);
+      // Find the volume within the datanode which holds that first storage.
+      DataNode dn = cluster.getDataNodes().get(0);
+      FsVolumeReferences volumeRefs =
+          dn.getFSDataset().getFsVolumeReferences();
+      final String newStorageId = DatanodeStorage.generateUuid();
+      try {
+        File currentDir = new File(volumeRefs.get(0).getBasePath(), "current");
+        File versionFile = new File(currentDir, "VERSION");
+        rewriteVersionFile(versionFile, newStorageId);
+      } finally {
+        volumeRefs.close();
+      }
+      final ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, TEST_PATH);
+      cluster.restartDataNodes();
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          cluster.getNamesystem().writeLock();
+          try {
+            Iterator<DatanodeStorageInfo> storageInfoIter =
+                cluster.getNamesystem().getBlockManager().
+                    getStorages(block.getLocalBlock()).iterator();
+            if (!storageInfoIter.hasNext()) {
+              LOG.info("Expected to find a storage for " +
+                      block.getBlockName() + ", but nothing was found.  " +
+                      "Continuing to wait.");
+              return false;
+            }
+            DatanodeStorageInfo info = storageInfoIter.next();
+            if (!newStorageId.equals(info.getStorageID())) {
+              LOG.info("Expected " + block.getBlockName() + " to " +
+                  "be in storage id " + newStorageId + ", but it " +
+                  "was in " + info.getStorageID() + ".  Continuing " +
+                  "to wait.");
+              return false;
+            }
+            LOG.info("Successfully found " + block.getBlockName() + " in " +
+                "be in storage id " + newStorageId);
+          } finally {
+            cluster.getNamesystem().writeUnlock();
+          }
+          return true;
+        }
+      }, 20, 100000);
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }


[14/37] hadoop git commit: YARN-3539. Updated timeline server documentation and marked REST APIs evolving. Contributed by Steve Loughran.

Posted by ji...@apache.org.
YARN-3539. Updated timeline server documentation and marked REST APIs evolving. Contributed by Steve Loughran.


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

Branch: refs/heads/HDFS-7240
Commit: fcd0702c10ce574b887280476aba63d6682d5271
Parents: 2463666
Author: Zhijie Shen <zj...@apache.org>
Authored: Tue May 12 21:12:18 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Tue May 12 21:13:22 2015 -0700

----------------------------------------------------------------------
 .../src/site/markdown/Compatibility.md          |    1 +
 hadoop-project/src/site/site.xml                |    1 +
 hadoop-yarn-project/CHANGES.txt                 |    3 +
 .../TimelineDelegationTokenResponse.java        |    4 +-
 .../api/records/timeline/TimelineDomain.java    |    4 +-
 .../api/records/timeline/TimelineDomains.java   |    4 +-
 .../api/records/timeline/TimelineEntities.java  |    4 +-
 .../api/records/timeline/TimelineEntity.java    |    4 +-
 .../api/records/timeline/TimelineEvent.java     |    4 +-
 .../api/records/timeline/TimelineEvents.java    |    6 +-
 .../records/timeline/TimelinePutResponse.java   |    6 +-
 .../yarn/api/records/timeline/package-info.java |    2 +
 .../hadoop/yarn/client/api/TimelineClient.java  |    4 +-
 .../client/api/impl/TimelineClientImpl.java     |    4 +-
 .../yarn/client/api/impl/package-info.java      |    2 +
 .../hadoop/yarn/client/api/package-info.java    |    2 +
 .../TimelineDelegationTokenIdentifier.java      |    4 +-
 .../client/TimelineDelegationTokenSelector.java |    4 +-
 .../yarn/server/webapp/dao/AppAttemptInfo.java  |    5 +
 .../yarn/server/webapp/dao/AppAttemptsInfo.java |    5 +
 .../hadoop/yarn/server/webapp/dao/AppInfo.java  |    5 +
 .../hadoop/yarn/server/webapp/dao/AppsInfo.java |    7 +-
 .../yarn/server/webapp/dao/ContainerInfo.java   |    5 +
 .../yarn/server/webapp/dao/ContainersInfo.java  |    5 +
 .../src/site/markdown/TimelineServer.md         | 1852 ++++++++++++++++--
 25 files changed, 1800 insertions(+), 147 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md b/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
index c058021..8326b5f 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
@@ -169,6 +169,7 @@ REST API compatibility corresponds to both the request (URLs) and responses to e
 * [NodeManager](../../hadoop-yarn/hadoop-yarn-site/NodeManagerRest.html)
 * [MR Application Master](../../hadoop-yarn/hadoop-yarn-site/MapredAppMasterRest.html)
 * [History Server](../../hadoop-yarn/hadoop-yarn-site/HistoryServerRest.html)
+* [Timeline Server v1 REST API](../../hadoop-yarn/hadoop-yarn-site/TimelineServer.html)
 
 #### Policy
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-project/src/site/site.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index 71defe5..7234881 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -134,6 +134,7 @@
       <item name="Introduction" href="hadoop-yarn/hadoop-yarn-site/WebServicesIntro.html"/>
       <item name="Resource Manager" href="hadoop-yarn/hadoop-yarn-site/ResourceManagerRest.html"/>
       <item name="Node Manager" href="hadoop-yarn/hadoop-yarn-site/NodeManagerRest.html"/>
+      <item name="Timeline Server" href="TimelineServer.html#Timeline_Server_REST_API_v1"/>
     </menu>
     
     <menu name="Hadoop Compatible File Systems" inherit="top">

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 5a858cf..59a3dd2 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -414,6 +414,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3489. RMServerUtils.validateResourceRequests should only obtain queue 
     info once. (Varun Saxena via wangda)
 
+    YARN-3539. Updated timeline server documentation and marked REST APIs evolving.
+    (Steve Loughran via zjshen)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDelegationTokenResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDelegationTokenResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDelegationTokenResponse.java
index 90d13a2..79fd359 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDelegationTokenResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDelegationTokenResponse.java
@@ -24,7 +24,7 @@ import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
 
 /**
  * The response of delegation token related request
@@ -32,7 +32,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @XmlRootElement(name = "delegationtoken")
 @XmlAccessorType(XmlAccessType.NONE)
 @Public
-@Unstable
+@Evolving
 public class TimelineDelegationTokenResponse {
 
   private String type;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDomain.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDomain.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDomain.java
index f8a7e60..8c08932 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDomain.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDomain.java
@@ -24,7 +24,7 @@ import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
 
 /**
  * <p>
@@ -43,7 +43,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @XmlRootElement(name = "domain")
 @XmlAccessorType(XmlAccessType.NONE)
 @Public
-@Unstable
+@Evolving
 public class TimelineDomain {
 
   private String id;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDomains.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDomains.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDomains.java
index 0da1a70..0d9e1f6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDomains.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineDomains.java
@@ -27,7 +27,7 @@ import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
 
 /**
  * The class that hosts a list of timeline domains.
@@ -35,7 +35,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @XmlRootElement(name = "domains")
 @XmlAccessorType(XmlAccessType.NONE)
 @Public
-@Unstable
+@Evolving
 public class TimelineDomains {
 
   private List<TimelineDomain> domains = new ArrayList<TimelineDomain>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntities.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntities.java
index 4a5c8f8..fb05365 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntities.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntities.java
@@ -27,7 +27,7 @@ import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
 
 /**
  * The class that hosts a list of timeline entities.
@@ -35,7 +35,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @XmlRootElement(name = "entities")
 @XmlAccessorType(XmlAccessType.NONE)
 @Public
-@Unstable
+@Evolving
 public class TimelineEntities {
 
   private List<TimelineEntity> entities =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntity.java
index 03ed4af..a43259b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntity.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEntity.java
@@ -33,7 +33,7 @@ import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
 
 /**
  * <p>
@@ -52,7 +52,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @XmlRootElement(name = "entity")
 @XmlAccessorType(XmlAccessType.NONE)
 @Public
-@Unstable
+@Evolving
 public class TimelineEntity implements Comparable<TimelineEntity> {
 
   private String entityType;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvent.java
index 0a09e78..73b2e72 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvent.java
@@ -28,7 +28,7 @@ import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
 
 /**
  * The class that contains the information of an event that is related to some
@@ -39,7 +39,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @XmlRootElement(name = "event")
 @XmlAccessorType(XmlAccessType.NONE)
 @Public
-@Unstable
+@Evolving
 public class TimelineEvent implements Comparable<TimelineEvent> {
 
   private long timestamp;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvents.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvents.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvents.java
index da62c27..8a70704 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvents.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineEvents.java
@@ -27,7 +27,7 @@ import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
 
 /**
  * The class that hosts a list of events, which are categorized according to
@@ -36,7 +36,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @XmlRootElement(name = "events")
 @XmlAccessorType(XmlAccessType.NONE)
 @Public
-@Unstable
+@Evolving
 public class TimelineEvents {
 
   private List<EventsOfOneEntity> allEvents =
@@ -93,7 +93,7 @@ public class TimelineEvents {
   @XmlRootElement(name = "events")
   @XmlAccessorType(XmlAccessType.NONE)
   @Public
-  @Unstable
+  @Evolving
   public static class EventsOfOneEntity {
 
     private String entityId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelinePutResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelinePutResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelinePutResponse.java
index abe106f..d683117 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelinePutResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelinePutResponse.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.yarn.api.records.timeline;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
 
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
@@ -36,7 +36,7 @@ import java.util.List;
 @XmlRootElement(name = "response")
 @XmlAccessorType(XmlAccessType.NONE)
 @Public
-@Unstable
+@Evolving
 public class TimelinePutResponse {
 
   private List<TimelinePutError> errors = new ArrayList<TimelinePutError>();
@@ -92,7 +92,7 @@ public class TimelinePutResponse {
   @XmlRootElement(name = "error")
   @XmlAccessorType(XmlAccessType.NONE)
   @Public
-  @Unstable
+  @Evolving
   public static class TimelinePutError {
 
     /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/package-info.java
index e91ddd4..3af14f2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/package-info.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/package-info.java
@@ -16,6 +16,8 @@
  * limitations under the License.
  */
 @InterfaceAudience.Public
+@InterfaceStability.Evolving
 package org.apache.hadoop.yarn.api.records.timeline;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
index 386602e..a3766f9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.service.AbstractService;
@@ -38,7 +38,7 @@ import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
  * number of conceptual entities.
  */
 @Public
-@Unstable
+@Evolving
 public abstract class TimelineClient extends AbstractService {
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
index 0ac72dc..76db4e2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
@@ -43,7 +43,7 @@ import org.apache.commons.cli.Options;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.SecurityUtil;
@@ -83,7 +83,7 @@ import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
 import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
 
 @Private
-@Unstable
+@Evolving
 public class TimelineClientImpl extends TimelineClient {
 
   private static final Log LOG = LogFactory.getLog(TimelineClientImpl.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/package-info.java
index 89b5b6b..9e2d911 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/package-info.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/package-info.java
@@ -16,6 +16,8 @@
  * limitations under the License.
  */
 @InterfaceAudience.Public
+@InterfaceStability.Evolving
 package org.apache.hadoop.yarn.client.api.impl;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/package-info.java
index efd7229..a37e3d9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/package-info.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/package-info.java
@@ -16,6 +16,8 @@
  * limitations under the License.
  */
 @InterfaceAudience.Public
+@InterfaceStability.Evolving
 package org.apache.hadoop.yarn.client.api;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/TimelineDelegationTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/TimelineDelegationTokenIdentifier.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/TimelineDelegationTokenIdentifier.java
index aa1bdec..1f01d0f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/TimelineDelegationTokenIdentifier.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/TimelineDelegationTokenIdentifier.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.token.Token;
@@ -31,7 +31,7 @@ import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 
 @Public
-@Unstable
+@Evolving
 public class TimelineDelegationTokenIdentifier extends YARNDelegationTokenIdentifier {
 
   public static final Text KIND_NAME = new Text("TIMELINE_DELEGATION_TOKEN");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/TimelineDelegationTokenSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/TimelineDelegationTokenSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/TimelineDelegationTokenSelector.java
index df1e84d..252dfa1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/TimelineDelegationTokenSelector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/TimelineDelegationTokenSelector.java
@@ -23,14 +23,14 @@ import java.util.Collection;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenSelector;
 
 @Public
-@Unstable
+@Evolving
 public class TimelineDelegationTokenSelector
     implements TokenSelector<TimelineDelegationTokenIdentifier> {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptInfo.java
index 92ebeb0..cd30bf0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptInfo.java
@@ -22,9 +22,14 @@ import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 
+@Public
+@Evolving
 @XmlRootElement(name = "appAttempt")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class AppAttemptInfo {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptsInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptsInfo.java
index d62bbe7..ae33b71 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptsInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptsInfo.java
@@ -24,6 +24,11 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+@Public
+@Evolving
 @XmlRootElement(name = "appAttempts")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class AppAttemptsInfo {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java
index 3eafb8c..5d9ba6f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java
@@ -24,11 +24,16 @@ import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.util.Times;
 
+@Public
+@Evolving
 @XmlRootElement(name = "app")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class AppInfo {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppsInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppsInfo.java
index f98b2ea..8d00642 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppsInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppsInfo.java
@@ -17,17 +17,22 @@
  */
 package org.apache.hadoop.yarn.server.webapp.dao;
 
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
 import java.util.ArrayList;
 
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
+@Public
+@Evolving
 @XmlRootElement(name = "apps")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class AppsInfo {
 
-  protected ArrayList<AppInfo> app = new ArrayList<AppInfo>();
+  protected ArrayList<AppInfo> app = new ArrayList<>();
 
   public AppsInfo() {
     // JAXB needs this

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerInfo.java
index 0d18e7a..d0d4df6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerInfo.java
@@ -22,10 +22,15 @@ import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
 import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.util.Times;
 
+@Public
+@Evolving
 @XmlRootElement(name = "container")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class ContainerInfo {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcd0702c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainersInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainersInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainersInfo.java
index 49767e0..545f84e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainersInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainersInfo.java
@@ -23,6 +23,11 @@ import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+@Public
+@Evolving
 @XmlRootElement(name = "containers")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class ContainersInfo {


[17/37] hadoop git commit: HDFS-6300. Prevent multiple balancers from running simultaneously (Contributed by Rakesh R)

Posted by ji...@apache.org.
HDFS-6300. Prevent multiple balancers from running simultaneously (Contributed by Rakesh R)


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

Branch: refs/heads/HDFS-7240
Commit: 065d8f2a34296b566e7ca541a284f7991212f14c
Parents: 92c38e4
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed May 13 17:27:34 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Wed May 13 17:27:34 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../hdfs/server/balancer/NameNodeConnector.java | 16 +++-
 .../hdfs/server/balancer/TestBalancer.java      | 77 ++++++++++++++++++++
 3 files changed, 92 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/065d8f2a/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 135b50c..4fabf97 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -826,6 +826,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-7916. 'reportBadBlocks' from datanodes to standby Node BPServiceActor
     goes for infinite loop (Rushabh S Shah  via kihwal)
 
+    HDFS-6300. Prevent multiple balancers from running simultaneously
+    (Rakesh R via vinayakumarb)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/065d8f2a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
index cf5f36f..2e4f214 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
@@ -219,12 +219,20 @@ public class NameNodeConnector implements Closeable {
    */
   private OutputStream checkAndMarkRunning() throws IOException {
     try {
-      final FSDataOutputStream out = fs.create(idPath);
+      if (fs.exists(idPath)) {
+        // try appending to it so that it will fail fast if another balancer is
+        // running.
+        IOUtils.closeStream(fs.append(idPath));
+        fs.delete(idPath, true);
+      }
+      final FSDataOutputStream fsout = fs.create(idPath, false);
+      // mark balancer idPath to be deleted during filesystem closure
+      fs.deleteOnExit(idPath);
       if (write2IdFile) {
-        out.writeBytes(InetAddress.getLocalHost().getHostName());
-        out.hflush();
+        fsout.writeBytes(InetAddress.getLocalHost().getHostName());
+        fsout.hflush();
       }
-      return out;
+      return fsout;
     } catch(RemoteException e) {
       if(AlreadyBeingCreatedException.class.getName().equals(e.getClassName())){
         return null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/065d8f2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index edffb82..e756f0b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@ -28,6 +28,7 @@ import static org.junit.Assume.assumeTrue;
 import java.io.File;
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.net.InetAddress;
 import java.net.URI;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
@@ -45,6 +46,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
@@ -1370,6 +1372,81 @@ public class TestBalancer {
       cluster.shutdown();
     }
   }
+
+  /**
+   * Test running many balancer simultaneously.
+   *
+   * Case-1: First balancer is running. Now, running second one should get
+   * "Another balancer is running. Exiting.." IOException and fail immediately
+   *
+   * Case-2: When running second balancer 'balancer.id' file exists but the
+   * lease doesn't exists. Now, the second balancer should run successfully.
+   */
+  @Test(timeout = 100000)
+  public void testManyBalancerSimultaneously() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    initConf(conf);
+    // add an empty node with half of the capacities(4 * CAPACITY) & the same
+    // rack
+    long[] capacities = new long[] { 4 * CAPACITY };
+    String[] racks = new String[] { RACK0 };
+    long newCapacity = 2 * CAPACITY;
+    String newRack = RACK0;
+    LOG.info("capacities = " + long2String(capacities));
+    LOG.info("racks      = " + Arrays.asList(racks));
+    LOG.info("newCapacity= " + newCapacity);
+    LOG.info("newRack    = " + newRack);
+    LOG.info("useTool    = " + false);
+    assertEquals(capacities.length, racks.length);
+    int numOfDatanodes = capacities.length;
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(capacities.length)
+        .racks(racks).simulatedCapacities(capacities).build();
+    try {
+      cluster.waitActive();
+      client = NameNodeProxies.createProxy(conf,
+          cluster.getFileSystem(0).getUri(), ClientProtocol.class).getProxy();
+
+      long totalCapacity = sum(capacities);
+
+      // fill up the cluster to be 30% full
+      final long totalUsedSpace = totalCapacity * 3 / 10;
+      createFile(cluster, filePath, totalUsedSpace / numOfDatanodes,
+          (short) numOfDatanodes, 0);
+      // start up an empty node with the same capacity and on the same rack
+      cluster.startDataNodes(conf, 1, true, null, new String[] { newRack },
+          new long[] { newCapacity });
+
+      // Case1: Simulate first balancer by creating 'balancer.id' file. It
+      // will keep this file until the balancing operation is completed.
+      FileSystem fs = cluster.getFileSystem(0);
+      final FSDataOutputStream out = fs
+          .create(Balancer.BALANCER_ID_PATH, false);
+      out.writeBytes(InetAddress.getLocalHost().getHostName());
+      out.hflush();
+      assertTrue("'balancer.id' file doesn't exist!",
+          fs.exists(Balancer.BALANCER_ID_PATH));
+
+      // start second balancer
+      final String[] args = { "-policy", "datanode" };
+      final Tool tool = new Cli();
+      tool.setConf(conf);
+      int exitCode = tool.run(args); // start balancing
+      assertEquals("Exit status code mismatches",
+          ExitStatus.IO_EXCEPTION.getExitCode(), exitCode);
+
+      // Case2: Release lease so that another balancer would be able to
+      // perform balancing.
+      out.close();
+      assertTrue("'balancer.id' file doesn't exist!",
+          fs.exists(Balancer.BALANCER_ID_PATH));
+      exitCode = tool.run(args); // start balancing
+      assertEquals("Exit status code mismatches",
+          ExitStatus.SUCCESS.getExitCode(), exitCode);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
   /**
    * @param args
    */


[34/37] hadoop git commit: HADOOP-10993. Dump java command line to *.out file (Contributed by Kengo Seki)

Posted by ji...@apache.org.
HADOOP-10993. Dump java command line to *.out file (Contributed by Kengo Seki)


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

Branch: refs/heads/HDFS-7240
Commit: ffbb574623c2a1dbcead201e9ae2dad3f77998d0
Parents: 0daa5ad
Author: Vinayakumar B <vi...@apache.org>
Authored: Thu May 14 15:24:35 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Thu May 14 15:24:35 2015 +0530

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt         |  3 +++
 .../hadoop-common/src/main/bin/hadoop-functions.sh      | 12 ++++++++++++
 2 files changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffbb5746/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 bf39c94..359a38b 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -215,6 +215,9 @@ Trunk (Unreleased)
     HADOOP-11590. Update sbin commands and documentation to use new --slaves
     option (aw)
 
+    HADOOP-10993. Dump java command line to *.out file
+    (Kengo Seki via vinayakumarb)
+
   BUG FIXES
 
     HADOOP-11473. test-patch says "-1 overall" even when all checks are +1

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffbb5746/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 85f8200..67e8870 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
@@ -1148,6 +1148,10 @@ function hadoop_java_exec
 
   hadoop_debug "Final CLASSPATH: ${CLASSPATH}"
   hadoop_debug "Final HADOOP_OPTS: ${HADOOP_OPTS}"
+  hadoop_debug "Final JAVA_HOME: ${JAVA_HOME}"
+  hadoop_debug "java: ${JAVA}"
+  hadoop_debug "Class name: ${class}"
+  hadoop_debug "Command line options: $*"
 
   export CLASSPATH
   #shellcheck disable=SC2086
@@ -1174,6 +1178,10 @@ function hadoop_start_daemon
 
   hadoop_debug "Final CLASSPATH: ${CLASSPATH}"
   hadoop_debug "Final HADOOP_OPTS: ${HADOOP_OPTS}"
+  hadoop_debug "Final JAVA_HOME: ${JAVA_HOME}"
+  hadoop_debug "java: ${JAVA}"
+  hadoop_debug "Class name: ${class}"
+  hadoop_debug "Command line options: $*"
 
   # this is for the non-daemon pid creation
   #shellcheck disable=SC2086
@@ -1300,6 +1308,10 @@ function hadoop_start_secure_daemon
 
   hadoop_debug "Final CLASSPATH: ${CLASSPATH}"
   hadoop_debug "Final HADOOP_OPTS: ${HADOOP_OPTS}"
+  hadoop_debug "Final JSVC_HOME: ${JSVC_HOME}"
+  hadoop_debug "jsvc: ${jsvc}"
+  hadoop_debug "Class name: ${class}"
+  hadoop_debug "Command line options: $*"
 
   #shellcheck disable=SC2086
   echo $$ > "${privpidfile}" 2>/dev/null


[10/37] hadoop git commit: YARN-3613. TestContainerManagerSecurity should init and start Yarn cluster in setup instead of individual methods. (nijel via kasha)

Posted by ji...@apache.org.
YARN-3613. TestContainerManagerSecurity should init and start Yarn cluster in setup instead of individual methods. (nijel via kasha)


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

Branch: refs/heads/HDFS-7240
Commit: fe0df596271340788095cb43a1944e19ac4c2cf7
Parents: 2f4b6d1
Author: Karthik Kambatla <ka...@apache.org>
Authored: Tue May 12 10:45:33 2015 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Tue May 12 10:45:33 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../server/TestContainerManagerSecurity.java    | 46 +++++---------------
 2 files changed, 15 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe0df596/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 131161f..5a858cf 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -230,6 +230,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3513. Remove unused variables in ContainersMonitorImpl and add debug
     log for overall resource usage by all containers. (Naganarasimha G R via devaraj)
 
+    YARN-3613. TestContainerManagerSecurity should init and start Yarn cluster in 
+    setup instead of individual methods. (nijel via kasha)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe0df596/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java
index f0dcb56..59bb6aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java
@@ -82,8 +82,6 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import com.google.common.io.ByteArrayDataInput;
-import com.google.common.io.ByteStreams;
 
 @RunWith(Parameterized.class)
 public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
@@ -105,10 +103,20 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
     testRootDir.mkdirs();
     httpSpnegoKeytabFile.deleteOnExit();
     getKdc().createPrincipal(httpSpnegoKeytabFile, httpSpnegoPrincipal);
+
+    yarnCluster =
+        new MiniYARNCluster(TestContainerManagerSecurity.class.getName(), 1, 1,
+            1);
+    yarnCluster.init(conf);
+    yarnCluster.start();
   }
  
   @After
   public void tearDown() {
+    if (yarnCluster != null) {
+      yarnCluster.stop();
+      yarnCluster = null;
+    }
     testRootDir.delete();
   }
 
@@ -144,11 +152,6 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
   
   @Test (timeout = 120000)
   public void testContainerManager() throws Exception {
-    try {
-      yarnCluster = new MiniYARNCluster(TestContainerManagerSecurity.class
-          .getName(), 1, 1, 1);
-      yarnCluster.init(conf);
-      yarnCluster.start();
       
       // TestNMTokens.
       testNMTokens(conf);
@@ -156,36 +159,11 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
       // Testing for container token tampering
       testContainerToken(conf);
       
-    } catch (Exception e) {
-      e.printStackTrace();
-      throw e;
-    } finally {
-      if (yarnCluster != null) {
-        yarnCluster.stop();
-        yarnCluster = null;
-      }
-    }
-  }
-
-  @Test (timeout = 120000)
-  public void testContainerManagerWithEpoch() throws Exception {
-    try {
-      yarnCluster = new MiniYARNCluster(TestContainerManagerSecurity.class
-          .getName(), 1, 1, 1);
-      yarnCluster.init(conf);
-      yarnCluster.start();
-
-      // Testing for container token tampering
+      // Testing for container token tampering with epoch
       testContainerTokenWithEpoch(conf);
 
-    } finally {
-      if (yarnCluster != null) {
-        yarnCluster.stop();
-        yarnCluster = null;
-      }
-    }
   }
-  
+
   private void testNMTokens(Configuration conf) throws Exception {
     NMTokenSecretManagerInRM nmTokenSecretManagerRM =
         yarnCluster.getResourceManager().getRMContext()


[22/37] hadoop git commit: YARN-3626. On Windows localized resources are not moved to the front of the classpath when they should be. Contributed by Craig Welch

Posted by ji...@apache.org.
YARN-3626. On Windows localized resources are not moved to the front of the classpath when they should be. Contributed by Craig Welch


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

Branch: refs/heads/HDFS-7240
Commit: 0f95921447ea547bdf9caf18f7fde46bc66031f8
Parents: 341a476
Author: Xuan <xg...@apache.org>
Authored: Wed May 13 13:10:53 2015 -0700
Committer: Xuan <xg...@apache.org>
Committed: Wed May 13 13:10:53 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/mapreduce/v2/util/MRApps.java |   7 +-
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |  10 ++
 .../launcher/ContainerLaunch.java               |  35 ++++-
 .../launcher/TestContainerLaunch.java           | 129 +++++++++++++++++++
 5 files changed, 181 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f959214/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
index 6b115b3..6d82104 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
@@ -241,7 +241,12 @@ public class MRApps extends Apps {
     boolean userClassesTakesPrecedence = 
       conf.getBoolean(MRJobConfig.MAPREDUCE_JOB_USER_CLASSPATH_FIRST, false);
 
-    String classpathEnvVar = 
+    if (userClassesTakesPrecedence) {
+      conf.set(YarnConfiguration.YARN_APPLICATION_CLASSPATH_PREPEND_DISTCACHE,
+        "true");
+    }
+
+    String classpathEnvVar =
       conf.getBoolean(MRJobConfig.MAPREDUCE_JOB_CLASSLOADER, false)
         ? Environment.APP_CLASSPATH.name() : Environment.CLASSPATH.name();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f959214/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index d61f7fc..e5d5ecf 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -480,6 +480,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3493. RM fails to come up with error "Failed to load/recover state"
     when mem settings are changed. (Jian He via wangda)
 
+    YARN-3626. On Windows localized resources are not moved to the front
+    of the classpath when they should be. (Craig Welch via xgong)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f959214/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 48a75c0..94f3e60 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -1249,6 +1249,16 @@ public class YarnConfiguration extends Configuration {
       + "application.classpath";
 
   /**
+   * Whether or not entries from the distributed cache should be preferred over
+   * the rest of the YARN CLASSPATH
+   */
+  public static final String YARN_APPLICATION_CLASSPATH_PREPEND_DISTCACHE =
+    YARN_PREFIX + "application.classpath.prepend.distcache";
+
+  public static final boolean
+    DEFAULT_YARN_APPLICATION_CLASSPATH_PREPEND_DISTCACHE = false;
+
+  /**
    * Default platform-agnostic CLASSPATH for YARN applications. A
    * comma-separated list of CLASSPATH entries. The parameter expansion marker
    * will be replaced with real parameter expansion marker ('%' for Windows and

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f959214/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
index 5a9229b..8433c21 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
@@ -726,8 +726,28 @@ public class ContainerLaunch implements Callable<Integer> {
     if (Shell.WINDOWS) {
       
       String inputClassPath = environment.get(Environment.CLASSPATH.name());
+
       if (inputClassPath != null && !inputClassPath.isEmpty()) {
-        StringBuilder newClassPath = new StringBuilder(inputClassPath);
+
+        //On non-windows, localized resources
+        //from distcache are available via the classpath as they were placed
+        //there but on windows they are not available when the classpath
+        //jar is created and so they "are lost" and have to be explicitly
+        //added to the classpath instead.  This also means that their position
+        //is lost relative to other non-distcache classpath entries which will
+        //break things like mapreduce.job.user.classpath.first.
+
+        boolean preferLocalizedJars = conf.getBoolean(
+          YarnConfiguration.YARN_APPLICATION_CLASSPATH_PREPEND_DISTCACHE,
+          YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH_PREPEND_DISTCACHE
+          );
+
+        boolean needsSeparator = false;
+        StringBuilder newClassPath = new StringBuilder();
+        if (!preferLocalizedJars) {
+          newClassPath.append(inputClassPath);
+          needsSeparator = true;
+        }
 
         // Localized resources do not exist at the desired paths yet, because the
         // container launch script has not run to create symlinks yet.  This
@@ -741,7 +761,12 @@ public class ContainerLaunch implements Callable<Integer> {
 
           for (String linkName : entry.getValue()) {
             // Append resource.
-            newClassPath.append(File.pathSeparator).append(pwd.toString())
+            if (needsSeparator) {
+              newClassPath.append(File.pathSeparator);
+            } else {
+              needsSeparator = true;
+            }
+            newClassPath.append(pwd.toString())
               .append(Path.SEPARATOR).append(linkName);
 
             // FileUtil.createJarWithClassPath must use File.toURI to convert
@@ -758,6 +783,12 @@ public class ContainerLaunch implements Callable<Integer> {
             }
           }
         }
+        if (preferLocalizedJars) {
+          if (needsSeparator) {
+            newClassPath.append(File.pathSeparator);
+          }
+          newClassPath.append(inputClassPath);
+        }
 
         // When the container launches, it takes the parent process's environment
         // and then adds/overwrites with the entries from the container launch

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f959214/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
index 4088c2a..efeeb1d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
@@ -39,6 +39,9 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.StringTokenizer;
+import java.util.jar.JarFile;
+import java.util.jar.Manifest;
 
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
@@ -65,6 +68,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Token;
@@ -78,12 +82,18 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
 import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerExitEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.ShellScriptBuilder;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
+import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
+import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
+import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
@@ -97,6 +107,17 @@ import org.junit.Test;
 
 public class TestContainerLaunch extends BaseContainerManagerTest {
 
+  protected Context distContext = new NMContext(new NMContainerTokenSecretManager(
+    conf), new NMTokenSecretManagerInNM(), null,
+    new ApplicationACLsManager(conf), new NMNullStateStoreService()) {
+    public int getHttpPort() {
+      return HTTP_PORT;
+    };
+    public NodeId getNodeId() {
+      return NodeId.newInstance("ahost", 1234);
+    };
+  };
+
   public TestContainerLaunch() throws UnsupportedFileSystemException {
     super();
   }
@@ -371,6 +392,114 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     }
   }
 
+  @Test
+  public void testPrependDistcache() throws Exception {
+
+    // Test is only relevant on Windows
+    Assume.assumeTrue(Shell.WINDOWS);
+
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    ApplicationId appId = ApplicationId.newInstance(0, 0);
+    ApplicationAttemptId appAttemptId =
+        ApplicationAttemptId.newInstance(appId, 1);
+
+    ContainerId cId = ContainerId.newContainerId(appAttemptId, 0);
+    Map<String, String> userSetEnv = new HashMap<String, String>();
+    userSetEnv.put(Environment.CONTAINER_ID.name(), "user_set_container_id");
+    userSetEnv.put(Environment.NM_HOST.name(), "user_set_NM_HOST");
+    userSetEnv.put(Environment.NM_PORT.name(), "user_set_NM_PORT");
+    userSetEnv.put(Environment.NM_HTTP_PORT.name(), "user_set_NM_HTTP_PORT");
+    userSetEnv.put(Environment.LOCAL_DIRS.name(), "user_set_LOCAL_DIR");
+    userSetEnv.put(Environment.USER.key(), "user_set_" +
+      Environment.USER.key());
+    userSetEnv.put(Environment.LOGNAME.name(), "user_set_LOGNAME");
+    userSetEnv.put(Environment.PWD.name(), "user_set_PWD");
+    userSetEnv.put(Environment.HOME.name(), "user_set_HOME");
+    userSetEnv.put(Environment.CLASSPATH.name(), "SYSTEM_CLPATH");
+    containerLaunchContext.setEnvironment(userSetEnv);
+    Container container = mock(Container.class);
+    when(container.getContainerId()).thenReturn(cId);
+    when(container.getLaunchContext()).thenReturn(containerLaunchContext);
+    when(container.getLocalizedResources()).thenReturn(null);
+    Dispatcher dispatcher = mock(Dispatcher.class);
+    EventHandler eventHandler = new EventHandler() {
+      public void handle(Event event) {
+        Assert.assertTrue(event instanceof ContainerExitEvent);
+        ContainerExitEvent exitEvent = (ContainerExitEvent) event;
+        Assert.assertEquals(ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
+            exitEvent.getType());
+      }
+    };
+    when(dispatcher.getEventHandler()).thenReturn(eventHandler);
+
+    Configuration conf = new Configuration();
+
+    ContainerLaunch launch = new ContainerLaunch(distContext, conf,
+        dispatcher, exec, null, container, dirsHandler, containerManager);
+
+    String testDir = System.getProperty("test.build.data",
+        "target/test-dir");
+    Path pwd = new Path(testDir);
+    List<Path> appDirs = new ArrayList<Path>();
+    List<String> containerLogs = new ArrayList<String>();
+
+    Map<Path, List<String>> resources = new HashMap<Path, List<String>>();
+    Path userjar = new Path("user.jar");
+    List<String> lpaths = new ArrayList<String>();
+    lpaths.add("userjarlink.jar");
+    resources.put(userjar, lpaths);
+
+    Path nmp = new Path(testDir);
+
+    launch.sanitizeEnv(
+      userSetEnv, pwd, appDirs, containerLogs, resources, nmp);
+
+    List<String> result =
+      getJarManifestClasspath(userSetEnv.get(Environment.CLASSPATH.name()));
+
+    Assert.assertTrue(result.size() > 1);
+    Assert.assertTrue(
+      result.get(result.size() - 1).endsWith("userjarlink.jar"));
+
+    //Now move userjar to the front
+
+    cId = ContainerId.newContainerId(appAttemptId, 1);
+    when(container.getContainerId()).thenReturn(cId);
+
+    conf.set(YarnConfiguration.YARN_APPLICATION_CLASSPATH_PREPEND_DISTCACHE,
+      "true");
+
+    launch = new ContainerLaunch(distContext, conf,
+        dispatcher, exec, null, container, dirsHandler, containerManager);
+
+    launch.sanitizeEnv(
+      userSetEnv, pwd, appDirs, containerLogs, resources, nmp);
+
+    result =
+      getJarManifestClasspath(userSetEnv.get(Environment.CLASSPATH.name()));
+
+    Assert.assertTrue(result.size() > 1);
+    Assert.assertTrue(
+      result.get(0).endsWith("userjarlink.jar"));
+
+  }
+
+  private static List<String> getJarManifestClasspath(String path)
+      throws Exception {
+    List<String> classpath = new ArrayList<String>();
+    JarFile jarFile = new JarFile(path);
+    Manifest manifest = jarFile.getManifest();
+    String cps = manifest.getMainAttributes().getValue("Class-Path");
+    StringTokenizer cptok = new StringTokenizer(cps);
+    while (cptok.hasMoreTokens()) {
+      String cpentry = cptok.nextToken();
+      classpath.add(cpentry);
+    }
+    return classpath;
+  }
+
   /**
    * See if environment variable is forwarded using sanitizeEnv.
    * @throws Exception


[03/37] hadoop git commit: HDFS-8362. Java Compilation Error in TestHdfsConfigFields.java (Contributed by Arshad Mohammad)

Posted by ji...@apache.org.
HDFS-8362. Java Compilation Error in TestHdfsConfigFields.java (Contributed by Arshad Mohammad)


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

Branch: refs/heads/HDFS-7240
Commit: 987abc99b0309a07f0a342746b2a5048d5c36ce0
Parents: 3d28611
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue May 12 12:09:13 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Tue May 12 12:09:13 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java   | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/987abc99/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 b67caed..7cff8d4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -738,6 +738,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8351. Remove namenode -finalize option from document. (aajisaka)
 
+    HDFS-8362. Java Compilation Error in TestHdfsConfigFields.java
+    (Arshad Mohammad via vinayakumarb)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/987abc99/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
index 0e75d81..a1f8a3c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hdfs.tools;
+package org.apache.hadoop.tools;
 
 import java.util.HashSet;
 


[18/37] hadoop git commit: HDFS-8143. Mover should exit after some retry when failed to move blocks. Contributed by surendra singh lilhore

Posted by ji...@apache.org.
HDFS-8143. Mover should exit after some retry when failed to move blocks.  Contributed by surendra singh lilhore


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

Branch: refs/heads/HDFS-7240
Commit: cdec12d1b84d444e13bf997c817643ec24aaa832
Parents: 065d8f2
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Wed May 13 11:57:49 2015 -0700
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Wed May 13 11:57:49 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  2 +
 .../apache/hadoop/hdfs/server/mover/Mover.java  | 30 ++++++++++++---
 .../hadoop/hdfs/server/mover/TestMover.java     | 39 +++++++++++++++++++-
 3 files changed, 65 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdec12d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index f8e9f3a..ae056fa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -353,6 +353,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final long    DFS_MOVER_MOVEDWINWIDTH_DEFAULT = 5400*1000L;
   public static final String  DFS_MOVER_MOVERTHREADS_KEY = "dfs.mover.moverThreads";
   public static final int     DFS_MOVER_MOVERTHREADS_DEFAULT = 1000;
+  public static final String  DFS_MOVER_RETRY_MAX_ATTEMPTS_KEY = "dfs.mover.retry.max.attempts";
+  public static final int     DFS_MOVER_RETRY_MAX_ATTEMPTS_DEFAULT = 10;
 
   public static final String  DFS_DATANODE_ADDRESS_KEY = "dfs.datanode.address";
   public static final int     DFS_DATANODE_DEFAULT_PORT = 50010;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdec12d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
index b32220c..0710f3e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
@@ -58,6 +58,7 @@ import java.io.InputStreamReader;
 import java.net.URI;
 import java.text.DateFormat;
 import java.util.*;
+import java.util.concurrent.atomic.AtomicInteger;
 
 @InterfaceAudience.Private
 public class Mover {
@@ -107,10 +108,12 @@ public class Mover {
   private final Dispatcher dispatcher;
   private final StorageMap storages;
   private final List<Path> targetPaths;
+  private final int retryMaxAttempts;
+  private final AtomicInteger retryCount;
 
   private final BlockStoragePolicy[] blockStoragePolicies;
 
-  Mover(NameNodeConnector nnc, Configuration conf) {
+  Mover(NameNodeConnector nnc, Configuration conf, AtomicInteger retryCount) {
     final long movedWinWidth = conf.getLong(
         DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_KEY,
         DFSConfigKeys.DFS_MOVER_MOVEDWINWIDTH_DEFAULT);
@@ -120,7 +123,10 @@ public class Mover {
     final int maxConcurrentMovesPerNode = conf.getInt(
         DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY,
         DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_DEFAULT);
-
+    this.retryMaxAttempts = conf.getInt(
+        DFSConfigKeys.DFS_MOVER_RETRY_MAX_ATTEMPTS_KEY,
+        DFSConfigKeys.DFS_MOVER_RETRY_MAX_ATTEMPTS_DEFAULT);
+    this.retryCount = retryCount;
     this.dispatcher = new Dispatcher(nnc, Collections.<String> emptySet(),
         Collections.<String> emptySet(), movedWinWidth, moverThreads, 0,
         maxConcurrentMovesPerNode, conf);
@@ -255,14 +261,27 @@ public class Mover {
      * @return whether there is still remaining migration work for the next
      *         round
      */
-    private boolean processNamespace() {
+    private boolean processNamespace() throws IOException {
       getSnapshottableDirs();
       boolean hasRemaining = false;
       for (Path target : targetPaths) {
         hasRemaining |= processPath(target.toUri().getPath());
       }
       // wait for pending move to finish and retry the failed migration
-      hasRemaining |= Dispatcher.waitForMoveCompletion(storages.targets.values());
+      boolean hasFailed = Dispatcher.waitForMoveCompletion(storages.targets
+          .values());
+      if (hasFailed) {
+        if (retryCount.get() == retryMaxAttempts) {
+          throw new IOException("Failed to move some block's after "
+              + retryMaxAttempts + " retries.");
+        } else {
+          retryCount.incrementAndGet();
+        }
+      } else {
+        // Reset retry count if no failure.
+        retryCount.set(0);
+      }
+      hasRemaining |= hasFailed;
       return hasRemaining;
     }
 
@@ -528,6 +547,7 @@ public class Mover {
             DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT) * 2000 +
         conf.getLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
             DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT) * 1000;
+    AtomicInteger retryCount = new AtomicInteger(0);
     LOG.info("namenodes = " + namenodes);
     
     List<NameNodeConnector> connectors = Collections.emptyList();
@@ -541,7 +561,7 @@ public class Mover {
         Iterator<NameNodeConnector> iter = connectors.iterator();
         while (iter.hasNext()) {
           NameNodeConnector nnc = iter.next();
-          final Mover m = new Mover(nnc, conf);
+          final Mover m = new Mover(nnc, conf, retryCount);
           final ExitStatus r = m.run();
 
           if (r == ExitStatus.SUCCESS) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cdec12d1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
index b2f9fce..f4bedab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
@@ -20,12 +20,14 @@ package org.apache.hadoop.hdfs.server.mover;
 import java.io.IOException;
 import java.net.URI;
 import java.util.*;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.collect.Maps;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -34,6 +36,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DBlock;
+import org.apache.hadoop.hdfs.server.balancer.ExitStatus;
 import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
 import org.apache.hadoop.hdfs.server.mover.Mover.MLocation;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
@@ -54,7 +57,7 @@ public class TestMover {
     final List<NameNodeConnector> nncs = NameNodeConnector.newNameNodeConnectors(
         nnMap, Mover.class.getSimpleName(), Mover.MOVER_ID_PATH, conf,
         NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS);
-    return new Mover(nncs.get(0), conf);
+    return new Mover(nncs.get(0), conf, new AtomicInteger(0));
   }
 
   @Test
@@ -324,4 +327,38 @@ public class TestMover {
       cluster.shutdown();
     }
   }
+
+  @Test
+  public void testMoverFailedRetry() throws Exception {
+    // HDFS-8147
+    final Configuration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_MOVER_RETRY_MAX_ATTEMPTS_KEY, "2");
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(3)
+        .storageTypes(
+            new StorageType[][] {{StorageType.DISK, StorageType.ARCHIVE},
+                {StorageType.DISK, StorageType.ARCHIVE},
+                {StorageType.DISK, StorageType.ARCHIVE}}).build();
+    try {
+      cluster.waitActive();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      final String file = "/testMoverFailedRetry";
+      // write to DISK
+      final FSDataOutputStream out = dfs.create(new Path(file), (short) 2);
+      out.writeChars("testMoverFailedRetry");
+      out.close();
+
+      // Delete block file so, block move will fail with FileNotFoundException
+      LocatedBlock lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
+      cluster.corruptBlockOnDataNodesByDeletingBlockFile(lb.getBlock());
+      // move to ARCHIVE
+      dfs.setStoragePolicy(new Path(file), "COLD");
+      int rc = ToolRunner.run(conf, new Mover.Cli(),
+          new String[] {"-p", file.toString()});
+      Assert.assertEquals("Movement should fail after some retry",
+          ExitStatus.IO_EXCEPTION.getExitCode(), rc);
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }


[21/37] hadoop git commit: YARN-2921. Fix MockRM/MockAM#waitForState sleep too long. (Tsuyoshi Ozawa via wangda)

Posted by ji...@apache.org.
YARN-2921. Fix MockRM/MockAM#waitForState sleep too long. (Tsuyoshi Ozawa via wangda)


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

Branch: refs/heads/HDFS-7240
Commit: 341a476812015d0d584b198b451ea9458645a47c
Parents: 93b770f
Author: Wangda Tan <wa...@apache.org>
Authored: Wed May 13 13:06:07 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed May 13 13:06:07 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../hadoop/yarn/event/AsyncDispatcher.java      |  3 +-
 .../yarn/server/resourcemanager/MockAM.java     | 33 ++++++++----
 .../yarn/server/resourcemanager/MockRM.java     | 55 ++++++++++++++------
 .../TestApplicationMasterService.java           |  1 +
 .../server/resourcemanager/TestRMRestart.java   | 15 +++++-
 .../applicationsmanager/TestAMRestart.java      | 20 +++----
 7 files changed, 90 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/341a4768/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 59a3dd2..d61f7fc 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -400,6 +400,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3629. NodeID is always printed as "null" in node manager initialization log.
     (nijel via devaraj)
 
+    YARN-2921. Fix MockRM/MockAM#waitForState sleep too long. 
+    (Tsuyoshi Ozawa via wangda)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/341a4768/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 d36d841..c54b9c7 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
@@ -141,7 +141,8 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
       synchronized (waitForDrained) {
         while (!drained && eventHandlingThread.isAlive()) {
           waitForDrained.wait(1000);
-          LOG.info("Waiting for AsyncDispatcher to drain.");
+          LOG.info("Waiting for AsyncDispatcher to drain. Thread state is :" +
+              eventHandlingThread.getState());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/341a4768/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
index 5c107aa..0e25360 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
@@ -43,10 +43,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.log4j.Logger;
 import org.junit.Assert;
 
 public class MockAM {
 
+  private static final Logger LOG = Logger.getLogger(MockAM.class);
+
   private volatile int responseId = 0;
   private final ApplicationAttemptId attemptId;
   private RMContext context;
@@ -73,18 +76,28 @@ public class MockAM {
   public void waitForState(RMAppAttemptState finalState) throws Exception {
     RMApp app = context.getRMApps().get(attemptId.getApplicationId());
     RMAppAttempt attempt = app.getRMAppAttempt(attemptId);
-    int timeoutSecs = 0;
+    final int timeoutMsecs = 40000;
+    final int minWaitMsecs = 1000;
+    final int waitMsPerLoop = 500;
+    int loop = 0;
     while (!finalState.equals(attempt.getAppAttemptState())
-        && timeoutSecs++ < 40) {
-      System.out
-          .println("AppAttempt : " + attemptId + " State is : " 
-              + attempt.getAppAttemptState()
-              + " Waiting for state : " + finalState);
-      Thread.sleep(1000);
+        && waitMsPerLoop * loop < timeoutMsecs) {
+      LOG.info("AppAttempt : " + attemptId + " State is : " +
+          attempt.getAppAttemptState() + " Waiting for state : " +
+          finalState);
+      Thread.yield();
+      Thread.sleep(waitMsPerLoop);
+      loop++;
+    }
+    int waitedMsecs = waitMsPerLoop * loop;
+    if (minWaitMsecs > waitedMsecs) {
+      Thread.sleep(minWaitMsecs - waitedMsecs);
+    }
+    LOG.info("Attempt State is : " + attempt.getAppAttemptState());
+    if (waitedMsecs >= timeoutMsecs) {
+      Assert.fail("Attempt state is not correct (timedout): expected: "
+          + finalState + " actual: " + attempt.getAppAttemptState());
     }
-    System.out.println("AppAttempt State is : " + attempt.getAppAttemptState());
-    Assert.assertEquals("AppAttempt state is not correct (timedout)",
-        finalState, attempt.getAppAttemptState());
   }
 
   public RegisterApplicationMasterResponse registerAppAttempt()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/341a4768/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.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/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index 63d6557..3469b0c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -65,6 +65,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabels
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
@@ -95,6 +96,7 @@ import org.junit.Assert;
 @SuppressWarnings("unchecked")
 public class MockRM extends ResourceManager {
 
+  static final Logger LOG = Logger.getLogger(MockRM.class);
   static final String ENABLE_WEBAPP = "mockrm.webapp.enabled";
 
   public MockRM() {
@@ -126,15 +128,23 @@ public class MockRM extends ResourceManager {
       throws Exception {
     RMApp app = getRMContext().getRMApps().get(appId);
     Assert.assertNotNull("app shouldn't be null", app);
-    int timeoutSecs = 0;
-    while (!finalState.equals(app.getState()) && timeoutSecs++ < 40) {
-      System.out.println("App : " + appId + " State is : " + app.getState()
-          + " Waiting for state : " + finalState);
-      Thread.sleep(2000);
+    final int timeoutMsecs = 80000;
+    final int waitMsPerLoop = 500;
+    int loop = 0;
+    while (!finalState.equals(app.getState()) &&
+        ((waitMsPerLoop * loop) < timeoutMsecs)) {
+      LOG.info("App : " + appId + " State is : " + app.getState() +
+          " Waiting for state : " + finalState);
+      Thread.yield();
+      Thread.sleep(waitMsPerLoop);
+      loop++;
+    }
+    int waitedMsecs = waitMsPerLoop * loop;
+    LOG.info("App State is : " + app.getState());
+    if (waitedMsecs >= timeoutMsecs) {
+      Assert.fail("App state is not correct (timedout): expected: " +
+          finalState + " actual: " + app.getState());
     }
-    System.out.println("App State is : " + app.getState());
-    Assert.assertEquals("App state is not correct (timedout)", finalState,
-        app.getState());
   }
   
   public void waitForState(ApplicationAttemptId attemptId, 
@@ -143,16 +153,27 @@ public class MockRM extends ResourceManager {
     RMApp app = getRMContext().getRMApps().get(attemptId.getApplicationId());
     Assert.assertNotNull("app shouldn't be null", app);
     RMAppAttempt attempt = app.getRMAppAttempt(attemptId);
-    int timeoutSecs = 0;
-    while (!finalState.equals(attempt.getAppAttemptState()) && timeoutSecs++ < 40) {
-      System.out.println("AppAttempt : " + attemptId 
-          + " State is : " + attempt.getAppAttemptState()
-          + " Waiting for state : " + finalState);
-      Thread.sleep(1000);
+    final int timeoutMsecs = 40000;
+    final int minWaitMsecs = 1000;
+    final int waitMsPerLoop = 10;
+    int loop = 0;
+    while (!finalState.equals(attempt.getAppAttemptState())
+        && waitMsPerLoop * loop < timeoutMsecs) {
+      LOG.info("AppAttempt : " + attemptId + " State is : " +
+          attempt.getAppAttemptState() + " Waiting for state : " + finalState);
+      Thread.yield();
+      Thread.sleep(waitMsPerLoop);
+      loop++;
+    }
+    int waitedMsecs = waitMsPerLoop * loop;
+    if (minWaitMsecs > waitedMsecs) {
+      Thread.sleep(minWaitMsecs - waitedMsecs);
+    }
+    LOG.info("Attempt State is : " + attempt.getAppAttemptState());
+    if (waitedMsecs >= timeoutMsecs) {
+      Assert.fail("Attempt state is not correct (timedout): expected: "
+          + finalState + " actual: " + attempt.getAppAttemptState());
     }
-    System.out.println("Attempt State is : " + attempt.getAppAttemptState());
-    Assert.assertEquals("Attempt state is not correct (timedout)", finalState,
-        attempt.getAppAttemptState());
   }
 
   public void waitForContainerAllocated(MockNM nm, ContainerId containerId)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/341a4768/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.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/TestApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java
index ca5c7a4..8c175b5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager;
 
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 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.fair.FairScheduler;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/341a4768/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
index c889446..628e9be 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
@@ -25,6 +25,7 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import com.google.common.base.Supplier;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -53,6 +54,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.service.Service.STATE;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
@@ -586,10 +588,19 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
             .getAppAttemptState());
     Assert.assertEquals(RMAppAttemptState.LAUNCHED,rmApp.getAppAttempts()
         .get(latestAppAttemptId).getAppAttemptState());
-    
+
     rm3.waitForState(latestAppAttemptId, RMAppAttemptState.FAILED);
     rm3.waitForState(rmApp.getApplicationId(), RMAppState.ACCEPTED);
-    Assert.assertEquals(4, rmApp.getAppAttempts().size());
+    final int maxRetry = 10;
+    final RMApp rmAppForCheck = rmApp;
+    GenericTestUtils.waitFor(
+        new Supplier<Boolean>() {
+          @Override
+          public Boolean get() {
+            return new Boolean(rmAppForCheck.getAppAttempts().size() == 4);
+          }
+        },
+        100, maxRetry);
     Assert.assertEquals(RMAppAttemptState.FAILED,
         rmApp.getAppAttempts().get(latestAppAttemptId).getAppAttemptState());
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/341a4768/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
index 7befba4..3387f41 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
@@ -592,7 +592,7 @@ public class TestAMRestart {
     rm2.stop();
   }
 
-  @Test (timeout = 50000)
+  @Test (timeout = 120000)
   public void testRMAppAttemptFailuresValidityInterval() throws Exception {
     YarnConfiguration conf = new YarnConfiguration();
     conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
@@ -612,10 +612,10 @@ public class TestAMRestart {
         new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());
     nm1.registerNode();
 
-    // set window size to a larger number : 20s
+    // set window size to a larger number : 60s
     // we will verify the app should be failed if
-    // two continuous attempts failed in 20s.
-    RMApp app = rm1.submitApp(200, 20000);
+    // two continuous attempts failed in 60s.
+    RMApp app = rm1.submitApp(200, 60000);
     
     MockAM am = MockRM.launchAM(app, rm1, nm1);
     // Fail current attempt normally
@@ -636,8 +636,8 @@ public class TestAMRestart {
     rm1.waitForState(app.getApplicationId(), RMAppState.FAILED);
 
     ControlledClock clock = new ControlledClock(new SystemClock());
-    // set window size to 6s
-    RMAppImpl app1 = (RMAppImpl)rm1.submitApp(200, 6000);;
+    // set window size to 10s
+    RMAppImpl app1 = (RMAppImpl)rm1.submitApp(200, 10000);;
     app1.setSystemClock(clock);
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
 
@@ -655,8 +655,8 @@ public class TestAMRestart {
     MockAM am2 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
     am2.waitForState(RMAppAttemptState.RUNNING);
 
-    // wait for 6 seconds
-    clock.setTime(System.currentTimeMillis() + 6*1000);
+    // wait for 10 seconds
+    clock.setTime(System.currentTimeMillis() + 10*1000);
     // Fail attempt2 normally
     nm1.nodeHeartbeat(am2.getApplicationAttemptId(),
       1, ContainerState.COMPLETE);
@@ -693,8 +693,8 @@ public class TestAMRestart {
     MockAM am4 =
         rm2.waitForNewAMToLaunchAndRegister(app1.getApplicationId(), 4, nm1);
 
-    // wait for 6 seconds
-    clock.setTime(System.currentTimeMillis() + 6*1000);
+    // wait for 10 seconds
+    clock.setTime(System.currentTimeMillis() + 10*1000);
     // Fail attempt4 normally
     nm1
       .nodeHeartbeat(am4.getApplicationAttemptId(), 1, ContainerState.COMPLETE);


[20/37] hadoop git commit: HADOOP-11966. Variable cygwin is undefined in hadoop-config.sh when executed through hadoop-daemon.sh. Contributed by Chris Nauroth.

Posted by ji...@apache.org.
HADOOP-11966. Variable cygwin is undefined in hadoop-config.sh when executed through hadoop-daemon.sh. Contributed by Chris Nauroth.


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

Branch: refs/heads/HDFS-7240
Commit: 93b770f7e778835a9dd76854b435c5250835d1a8
Parents: f9a46a0
Author: cnauroth <cn...@apache.org>
Authored: Wed May 13 12:25:06 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Wed May 13 12:25:06 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/93b770f7/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 2fbecbb..207d144 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -703,6 +703,9 @@ Release 2.7.1 - UNRELEASED
     HADOOP-11663. Remove description about Java 6 from docs.
     (Masatake Iwasaki via aajisaka)
 
+    HADOOP-11966. Variable cygwin is undefined in hadoop-config.sh when executed
+    through hadoop-daemon.sh. (cnauroth)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES


[35/37] hadoop git commit: HDFS-8150. Make getFileChecksum fail for blocks under construction (Contributed by J.Andreina)

Posted by ji...@apache.org.
HDFS-8150. Make getFileChecksum fail for blocks under construction (Contributed by J.Andreina)


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

Branch: refs/heads/HDFS-7240
Commit: def9136e0259e118e6fd7b656260765d28ac9ae6
Parents: ffbb574
Author: Vinayakumar B <vi...@apache.org>
Authored: Thu May 14 15:54:51 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Thu May 14 15:54:51 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt      |  3 +++
 .../java/org/apache/hadoop/hdfs/DFSClient.java   |  8 ++++++++
 .../apache/hadoop/hdfs/TestGetFileChecksum.java  | 19 +++++++++++++++++++
 .../snapshot/TestSnapshotFileLength.java         | 17 +++++++++++++----
 4 files changed, 43 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/def9136e/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 0e6508b..4df18ec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -758,6 +758,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-7728. Avoid updating quota usage while loading edits.
     (Jing Zhao via wheat9)
 
+    HDFS-8150. Make getFileChecksum fail for blocks under construction
+    (J.Andreina via vinayakumarb)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/def9136e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 191ebc9..7908451 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -1872,6 +1872,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     if (null == blockLocations) {
       throw new FileNotFoundException("File does not exist: " + src);
     }
+    if (blockLocations.isUnderConstruction()) {
+      throw new IOException("Fail to get checksum, since file " + src
+          + " is under construction.");
+    }
     List<LocatedBlock> locatedblocks = blockLocations.getLocatedBlocks();
     final DataOutputBuffer md5out = new DataOutputBuffer();
     int bytesPerCRC = -1;
@@ -1891,6 +1895,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
         if (null == blockLocations) {
           throw new FileNotFoundException("File does not exist: " + src);
         }
+        if (blockLocations.isUnderConstruction()) {
+          throw new IOException("Fail to get checksum, since file " + src
+              + " is under construction.");
+        }
         locatedblocks = blockLocations.getLocatedBlocks();
         refetchBlocks = false;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/def9136e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetFileChecksum.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetFileChecksum.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetFileChecksum.java
index 0e56ba7..814261f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetFileChecksum.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetFileChecksum.java
@@ -17,7 +17,12 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.Path;
 import org.junit.After;
@@ -68,6 +73,20 @@ public class TestGetFileChecksum {
   }
 
   @Test
+  public void testGetFileChecksumForBlocksUnderConstruction() {
+    try {
+      FSDataOutputStream file = dfs.create(new Path("/testFile"));
+      file.write("Performance Testing".getBytes());
+      dfs.getFileChecksum(new Path("/testFile"));
+      fail("getFileChecksum should fail for files "
+          + "with blocks under construction");
+    } catch (IOException ie) {
+      Assert.assertTrue(ie.getMessage().contains(
+          "Fail to get checksum, since file /testFile "
+              + "is under construction."));
+    }
+  }
+  @Test
   public void testGetFileChecksum() throws Exception {
     testGetFileChecksum(new Path("/foo"), BLOCKSIZE / 4);
     testGetFileChecksum(new Path("/bar"), BLOCKSIZE / 4 - 1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/def9136e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotFileLength.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotFileLength.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotFileLength.java
index d53140f..90e499d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotFileLength.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotFileLength.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 import java.io.ByteArrayOutputStream;
+import java.io.IOException;
 import java.io.PrintStream;
 
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -32,6 +33,8 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.not;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -111,8 +114,16 @@ public class TestSnapshotFileLength {
     // Append to the file.
     FSDataOutputStream out = hdfs.append(file1);
     // Nothing has been appended yet. All checksums should still be equal.
-    assertThat("file and snapshot checksums (open for append) are not equal",
-        hdfs.getFileChecksum(file1), is(snapChksum1));
+    // HDFS-8150:Fetching checksum for file under construction should fail
+    try {
+      hdfs.getFileChecksum(file1);
+      fail("getFileChecksum should fail for files "
+          + "with blocks under construction");
+    } catch (IOException ie) {
+      assertTrue(ie.getMessage().contains(
+          "Fail to get checksum, since file " + file1
+              + " is under construction."));
+    }
     assertThat("snapshot checksum (post-open for append) has changed",
         hdfs.getFileChecksum(file1snap1), is(snapChksum1));
     try {
@@ -122,8 +133,6 @@ public class TestSnapshotFileLength {
       assertThat("Wrong data size in snapshot.",
           dataFromSnapshot.length, is(origLen));
       // Verify that checksum didn't change
-      assertThat("snapshot file checksum (pre-close) has changed",
-          hdfs.getFileChecksum(file1), is(snapChksum1));
       assertThat("snapshot checksum (post-append) has changed",
           hdfs.getFileChecksum(file1snap1), is(snapChksum1));
     } finally {


[24/37] hadoop git commit: YARN-3521. Support return structured NodeLabel objects in REST API (Sunil G via wangda)

Posted by ji...@apache.org.
YARN-3521. Support return structured NodeLabel objects in REST API (Sunil G via wangda)


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

Branch: refs/heads/HDFS-7240
Commit: 7f19e7a2549a098236d06b29b7076bb037533f05
Parents: d4f53fc
Author: Wangda Tan <wa...@apache.org>
Authored: Wed May 13 13:43:17 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed May 13 13:43:17 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   8 +-
 .../resourcemanager/webapp/NodeIDsInfo.java     |   2 +
 .../resourcemanager/webapp/RMWebServices.java   |  71 +++---
 .../webapp/dao/LabelsToNodesInfo.java           |   6 +-
 .../webapp/dao/NodeLabelInfo.java               |  86 +++++++
 .../webapp/dao/NodeLabelsInfo.java              |  56 ++++-
 .../webapp/dao/NodeToLabelsEntry.java           |  54 +++++
 .../webapp/dao/NodeToLabelsEntryList.java       |  41 ++++
 .../webapp/dao/NodeToLabelsInfo.java            |  12 +-
 .../webapp/TestRMWebServicesNodeLabels.java     | 225 ++++++++++++-------
 10 files changed, 427 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f19e7a2/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 16c2dd9..38f5e81 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -233,6 +233,11 @@ Release 2.8.0 - UNRELEASED
     YARN-3613. TestContainerManagerSecurity should init and start Yarn cluster in 
     setup instead of individual methods. (nijel via kasha)
 
+    YARN-3579. CommonNodeLabelsManager should support NodeLabel instead of string 
+    label name when getting node-to-label/label-to-label mappings. (Sunil G via wangda)
+
+    YARN-3521. Support return structured NodeLabel objects in REST API (Sunil G via wangda)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not
@@ -420,9 +425,6 @@ Release 2.7.1 - UNRELEASED
     YARN-3539. Updated timeline server documentation and marked REST APIs evolving.
     (Steve Loughran via zjshen)
 
-    YARN-3579. CommonNodeLabelsManager should support NodeLabel instead of string 
-    label name when getting node-to-label/label-to-label mappings. (Sunil G via wangda)
-
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f19e7a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeIDsInfo.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/NodeIDsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeIDsInfo.java
index 39d636d..c23b02a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeIDsInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeIDsInfo.java
@@ -23,6 +23,7 @@ import java.util.List;
 
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 
 @XmlRootElement(name = "labelsToNodesInfo")
@@ -32,6 +33,7 @@ public class NodeIDsInfo {
   /**
    * Set doesn't support default no arg constructor which is req by JAXB
    */
+  @XmlElement(name="nodes")
   protected ArrayList<String> nodeIDsList = new ArrayList<String>();
 
   public NodeIDsInfo() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f19e7a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 4ce2b54..a0a3123 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -93,6 +93,7 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -135,8 +136,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LabelsToNodesInf
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LocalResourceInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntry;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntryList;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerInfo;
@@ -796,18 +800,18 @@ public class RMWebServices {
   @GET
   @Path("/get-node-to-labels")
   @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
-  public NodeToLabelsInfo getNodeToLabels(@Context HttpServletRequest hsr) 
-    throws IOException {
+  public NodeToLabelsInfo getNodeToLabels(@Context HttpServletRequest hsr)
+      throws IOException {
     init();
 
     NodeToLabelsInfo ntl = new NodeToLabelsInfo();
     HashMap<String, NodeLabelsInfo> ntlMap = ntl.getNodeToLabels();
-    Map<NodeId, Set<String>> nodeIdToLabels =   
-      rm.getRMContext().getNodeLabelManager().getNodeLabels();
-      
+    Map<NodeId, Set<String>> nodeIdToLabels = rm.getRMContext()
+        .getNodeLabelManager().getNodeLabels();
+
     for (Map.Entry<NodeId, Set<String>> nitle : nodeIdToLabels.entrySet()) {
-      ntlMap.put(nitle.getKey().toString(), 
-        new NodeLabelsInfo(nitle.getValue()));
+      ntlMap.put(nitle.getKey().toString(),
+          new NodeLabelsInfo(nitle.getValue()));
     }
 
     return ntl;
@@ -821,7 +825,7 @@ public class RMWebServices {
     init();
 
     LabelsToNodesInfo lts = new LabelsToNodesInfo();
-    Map<String, NodeIDsInfo> ltsMap = lts.getLabelsToNodes();
+    Map<NodeLabelInfo, NodeIDsInfo> ltsMap = lts.getLabelsToNodes();
     Map<String, Set<NodeId>> labelsToNodeId = null;
     if (labels == null || labels.size() == 0) {
       labelsToNodeId =
@@ -836,7 +840,8 @@ public class RMWebServices {
       for (NodeId nodeId : entry.getValue()) {
         nodeIdStrList.add(nodeId.toString());
       }
-      ltsMap.put(entry.getKey(), new NodeIDsInfo(nodeIdStrList));
+      ltsMap.put(new NodeLabelInfo(entry.getKey()), new NodeIDsInfo(
+          nodeIdStrList));
     }
     return lts;
   }
@@ -844,16 +849,15 @@ public class RMWebServices {
   @POST
   @Path("/replace-node-to-labels")
   @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
-  public Response replaceLabelsOnNodes(final NodeToLabelsInfo newNodeToLabels,
+  public Response replaceLabelsOnNodes(final NodeToLabelsEntryList newNodeToLabels,
       @Context HttpServletRequest hsr) throws IOException {
     Map<NodeId, Set<String>> nodeIdToLabels =
         new HashMap<NodeId, Set<String>>();
 
-    for (Map.Entry<String, NodeLabelsInfo> nitle : newNodeToLabels
-        .getNodeToLabels().entrySet()) {
+    for (NodeToLabelsEntry nitle : newNodeToLabels.getNodeToLabels()) {
       nodeIdToLabels.put(
-          ConverterUtils.toNodeIdWithDefaultPort(nitle.getKey()),
-          new HashSet<String>(nitle.getValue().getNodeLabels()));
+          ConverterUtils.toNodeIdWithDefaultPort(nitle.getNodeId()),
+          new HashSet<String>(nitle.getNodeLabels()));
     }
 
     return replaceLabelsOnNode(nodeIdToLabels, hsr, "/replace-node-to-labels");
@@ -862,16 +866,18 @@ public class RMWebServices {
   @POST
   @Path("/nodes/{nodeId}/replace-labels")
   @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
-  public Response replaceLabelsOnNode(NodeLabelsInfo newNodeLabelsInfo,
+  public Response replaceLabelsOnNode(
+      @QueryParam("labels") Set<String> newNodeLabelsName,
       @Context HttpServletRequest hsr, @PathParam("nodeId") String nodeId)
       throws Exception {
     NodeId nid = ConverterUtils.toNodeIdWithDefaultPort(nodeId);
     Map<NodeId, Set<String>> newLabelsForNode =
         new HashMap<NodeId, Set<String>>();
     newLabelsForNode.put(nid,
-        new HashSet<String>(newNodeLabelsInfo.getNodeLabels()));
+        new HashSet<String>(newNodeLabelsName));
 
-    return replaceLabelsOnNode(newLabelsForNode, hsr, "/nodes/nodeid/replace-labels");
+    return replaceLabelsOnNode(newLabelsForNode, hsr,
+        "/nodes/nodeid/replace-labels");
   }
 
   private Response replaceLabelsOnNode(
@@ -909,9 +915,9 @@ public class RMWebServices {
     throws IOException {
     init();
 
-    NodeLabelsInfo ret = 
-      new NodeLabelsInfo(rm.getRMContext().getNodeLabelManager()
-        .getClusterNodeLabelNames());
+    List<NodeLabel> nodeLabels = rm.getRMContext().getNodeLabelManager()
+        .getClusterNodeLabels();
+    NodeLabelsInfo ret = new NodeLabelsInfo(nodeLabels);
 
     return ret;
   }
@@ -937,8 +943,7 @@ public class RMWebServices {
     }
     
     rm.getRMContext().getNodeLabelManager()
-        .addToCluserNodeLabelsWithDefaultExclusivity(new HashSet<String>(
-          newNodeLabels.getNodeLabels()));
+        .addToCluserNodeLabels(newNodeLabels.getNodeLabels());
             
     return Response.status(Status.OK).build();
 
@@ -947,29 +952,29 @@ public class RMWebServices {
   @POST
   @Path("/remove-node-labels")
   @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
-  public Response removeFromCluserNodeLabels(final NodeLabelsInfo oldNodeLabels,
-      @Context HttpServletRequest hsr)
-      throws Exception {
+  public Response removeFromCluserNodeLabels(
+      @QueryParam("labels") Set<String> oldNodeLabels,
+      @Context HttpServletRequest hsr) throws Exception {
     init();
 
     UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true);
     if (callerUGI == null) {
       String msg = "Unable to obtain user name, user not authenticated for"
-        + " post to .../remove-node-labels";
+          + " post to .../remove-node-labels";
       throw new AuthorizationException(msg);
     }
     if (!rm.getRMContext().getNodeLabelManager().checkAccess(callerUGI)) {
       String msg = "User " + callerUGI.getShortUserName() + " not authorized"
-        + " for post to .../remove-node-labels ";
+          + " for post to .../remove-node-labels ";
       throw new AuthorizationException(msg);
     }
-    
-    rm.getRMContext().getNodeLabelManager()
-        .removeFromClusterNodeLabels(new HashSet<String>(
-          oldNodeLabels.getNodeLabels()));
-            
-    return Response.status(Status.OK).build();
 
+    rm.getRMContext()
+        .getNodeLabelManager()
+        .removeFromClusterNodeLabels(
+            new HashSet<String>(oldNodeLabels));
+
+    return Response.status(Status.OK).build();
   }
   
   @GET

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f19e7a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/LabelsToNodesInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/LabelsToNodesInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/LabelsToNodesInfo.java
index 625fedd..41dd410 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/LabelsToNodesInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/LabelsToNodesInfo.java
@@ -31,13 +31,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.NodeIDsInfo;
 @XmlAccessorType(XmlAccessType.FIELD)
 public class LabelsToNodesInfo {
 
-  protected Map<String, NodeIDsInfo> labelsToNodes =
-    new HashMap<String, NodeIDsInfo>();
+  protected Map<NodeLabelInfo, NodeIDsInfo> labelsToNodes =
+    new HashMap<NodeLabelInfo, NodeIDsInfo>();
 
   public LabelsToNodesInfo() {
   } // JAXB needs this
 
-  public Map<String, NodeIDsInfo> getLabelsToNodes() {
+  public Map<NodeLabelInfo, NodeIDsInfo> getLabelsToNodes() {
    return labelsToNodes;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f19e7a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeLabelInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeLabelInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeLabelInfo.java
new file mode 100644
index 0000000..e507c46
--- /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/dao/NodeLabelInfo.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.yarn.api.records.NodeLabel;
+
+@XmlRootElement(name = "nodeLabelInfo")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class NodeLabelInfo {
+
+  private String name;
+  private boolean exclusivity;
+
+  public NodeLabelInfo() {
+    // JAXB needs this
+  }
+
+  public NodeLabelInfo(String name) {
+    this.name = name;
+    this.exclusivity = true;
+  }
+
+  public NodeLabelInfo(String name, boolean exclusivity) {
+    this.name = name;
+    this.exclusivity = exclusivity;
+  }
+
+  public NodeLabelInfo(NodeLabel label) {
+    this.name = label.getName();
+    this.exclusivity = label.isExclusive();
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public boolean getExclusivity() {
+    return exclusivity;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    NodeLabelInfo other = (NodeLabelInfo) obj;
+    if (!getName().equals(other.getName())) {
+      return false;
+    }
+    if (getExclusivity() != other.getExclusivity()) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    return (getName().hashCode() << 16) + (getExclusivity() ? 1 : 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f19e7a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeLabelsInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeLabelsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeLabelsInfo.java
index 1cb895a..2c3a8a5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeLabelsInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeLabelsInfo.java
@@ -22,31 +22,63 @@ import java.util.*;
 
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 
+import org.apache.hadoop.yarn.api.records.NodeLabel;
+
 @XmlRootElement(name = "nodeLabelsInfo")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class NodeLabelsInfo {
 
-  protected ArrayList<String> nodeLabels = new ArrayList<String>();
+  @XmlElement(name = "nodeLabelInfo")
+  private ArrayList<NodeLabelInfo> nodeLabelsInfo =
+    new ArrayList<NodeLabelInfo>();
 
   public NodeLabelsInfo() {
-  } // JAXB needs this
-  
-  public NodeLabelsInfo(ArrayList<String> nodeLabels) {
-   this.nodeLabels = nodeLabels; 
+    // JAXB needs this
   }
-  
-  public NodeLabelsInfo(Set<String> nodeLabelsSet) {
-   this.nodeLabels = new ArrayList<String>(nodeLabelsSet); 
+
+  public NodeLabelsInfo(ArrayList<NodeLabelInfo> nodeLabels) {
+    this.nodeLabelsInfo = nodeLabels;
+  }
+
+  public NodeLabelsInfo(List<NodeLabel> nodeLabels) {
+    this.nodeLabelsInfo = new ArrayList<NodeLabelInfo>();
+    for (NodeLabel label : nodeLabels) {
+      this.nodeLabelsInfo.add(new NodeLabelInfo(label));
+    }
   }
   
-  public ArrayList<String> getNodeLabels() {
+  public NodeLabelsInfo(Set<String> nodeLabelsName) {
+    this.nodeLabelsInfo = new ArrayList<NodeLabelInfo>();
+    for (String labelName : nodeLabelsName) {
+      this.nodeLabelsInfo.add(new NodeLabelInfo(labelName));
+    }
+  }
+
+  public ArrayList<NodeLabelInfo> getNodeLabelsInfo() {
+    return nodeLabelsInfo;
+  }
+
+  public Set<NodeLabel> getNodeLabels() {
+    Set<NodeLabel> nodeLabels = new HashSet<NodeLabel>();
+    for (NodeLabelInfo label : nodeLabelsInfo) {
+      nodeLabels.add(NodeLabel.newInstance(label.getName(),
+          label.getExclusivity()));
+    }
     return nodeLabels;
   }
   
-  public void setNodeLabels(ArrayList<String> nodeLabels) {
-    this.nodeLabels = nodeLabels; 
+  public List<String> getNodeLabelsName() {
+    ArrayList<String> nodeLabelsName = new ArrayList<String>();
+    for (NodeLabelInfo label : nodeLabelsInfo) {
+      nodeLabelsName.add(label.getName());
+    }
+    return nodeLabelsName;
+  }
+
+  public void setNodeLabelsInfo(ArrayList<NodeLabelInfo> nodeLabelInfo) {
+    this.nodeLabelsInfo = nodeLabelInfo;
   }
-  
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f19e7a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeToLabelsEntry.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeToLabelsEntry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeToLabelsEntry.java
new file mode 100644
index 0000000..702d6f0
--- /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/dao/NodeToLabelsEntry.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
+
+import java.util.*;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlElement;
+
+@XmlRootElement(name = "nodeToLabelsEntry")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class NodeToLabelsEntry {
+
+  @XmlElement(name = "nodeId")
+  private String nodeId;
+
+  @XmlElement(name = "labels")
+  private ArrayList<String> labels = new ArrayList<String>();
+
+  public NodeToLabelsEntry() {
+    // JAXB needs this
+  }
+
+  public NodeToLabelsEntry(String nodeId, ArrayList<String> labels) {
+    this.nodeId = nodeId;
+    this.labels = labels;
+  }
+
+  public String getNodeId() {
+    return nodeId;
+  }
+
+  public ArrayList<String> getNodeLabels() {
+    return labels;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f19e7a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeToLabelsEntryList.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeToLabelsEntryList.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeToLabelsEntryList.java
new file mode 100644
index 0000000..dc9543e
--- /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/dao/NodeToLabelsEntryList.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
+
+import java.util.*;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlRootElement(name = "nodeToLabelsName")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class NodeToLabelsEntryList {
+
+  protected ArrayList<NodeToLabelsEntry> nodeToLabels =
+      new ArrayList<NodeToLabelsEntry>();
+
+  public NodeToLabelsEntryList() {
+    // JAXB needs this
+  }
+
+  public ArrayList<NodeToLabelsEntry> getNodeToLabels() {
+    return nodeToLabels;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f19e7a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeToLabelsInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeToLabelsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeToLabelsInfo.java
index f2e6441..0b6e4bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeToLabelsInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeToLabelsInfo.java
@@ -28,14 +28,14 @@ import javax.xml.bind.annotation.XmlRootElement;
 @XmlAccessorType(XmlAccessType.FIELD)
 public class NodeToLabelsInfo {
 
-  protected HashMap<String, NodeLabelsInfo> nodeToLabels = 
-    new HashMap<String, NodeLabelsInfo>();
+  private HashMap<String, NodeLabelsInfo> nodeToLabels =
+      new HashMap<String, NodeLabelsInfo>();
 
   public NodeToLabelsInfo() {
-  } // JAXB needs this
-  
+    // JAXB needs this
+  }
+
   public HashMap<String, NodeLabelsInfo> getNodeToLabels() {
-   return nodeToLabels; 
+    return nodeToLabels;
   }
-  
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f19e7a2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.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/TestRMWebServicesNodeLabels.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
index 2d5518d..54fe00d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.io.StringWriter;
+import java.util.ArrayList;
 
 import javax.ws.rs.core.MediaType;
 
@@ -34,13 +35,14 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LabelsToNodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntry;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntryList;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
-import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
-import org.codehaus.jettison.json.JSONObject;
 import org.junit.Test;
 
 import com.google.inject.Guice;
@@ -113,15 +115,15 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
     WebResource r = resource();
 
     ClientResponse response;
-    JSONObject json;
-    JSONArray jarr;
 
     // Add a label
+    NodeLabelsInfo nlsifo = new NodeLabelsInfo();
+    nlsifo.getNodeLabelsInfo().add(new NodeLabelInfo("a"));
     response =
         r.path("ws").path("v1").path("cluster")
             .path("add-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\":\"a\"}", MediaType.APPLICATION_JSON)
+            .entity(toJson(nlsifo, NodeLabelsInfo.class), MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
 
     // Verify
@@ -130,15 +132,18 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("get-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    json = response.getEntity(JSONObject.class);
-    assertEquals("a", json.getString("nodeLabels"));
+    nlsifo = response.getEntity(NodeLabelsInfo.class);
+    assertEquals("a", nlsifo.getNodeLabelsInfo().get(0).getName());
+    assertEquals(1, nlsifo.getNodeLabels().size());
     
     // Add another
+    nlsifo = new NodeLabelsInfo();
+    nlsifo.getNodeLabelsInfo().add(new NodeLabelInfo("b"));
     response =
         r.path("ws").path("v1").path("cluster")
             .path("add-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\":\"b\"}", MediaType.APPLICATION_JSON)
+            .entity(toJson(nlsifo, NodeLabelsInfo.class), MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
 
     // Verify
@@ -147,43 +152,45 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("get-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    json = response.getEntity(JSONObject.class);
-    jarr = json.getJSONArray("nodeLabels");
-    assertEquals(2, jarr.length());
+    nlsifo = response.getEntity(NodeLabelsInfo.class);
+    assertEquals(2, nlsifo.getNodeLabels().size());
     
     // Add labels to a node
+    MultivaluedMapImpl params = new MultivaluedMapImpl();
+    params.add("labels", "a");
     response =
         r.path("ws").path("v1").path("cluster")
             .path("nodes").path("nid:0")
             .path("replace-labels")
             .queryParam("user.name", userName)
+            .queryParams(params)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\": [\"a\"]}",
-              MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
     LOG.info("posted node nodelabel");
 
     // Add labels to another node
+    params = new MultivaluedMapImpl();
+    params.add("labels", "b");
     response =
         r.path("ws").path("v1").path("cluster")
             .path("nodes").path("nid1:0")
             .path("replace-labels")
             .queryParam("user.name", userName)
+            .queryParams(params)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\": [\"b\"]}",
-              MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
     LOG.info("posted node nodelabel");
 
     // Add labels to another node
+    params = new MultivaluedMapImpl();
+    params.add("labels", "b");
     response =
         r.path("ws").path("v1").path("cluster")
             .path("nodes").path("nid2:0")
             .path("replace-labels")
             .queryParam("user.name", userName)
+            .queryParams(params)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\": [\"b\"]}",
-              MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
     LOG.info("posted node nodelabel");
 
@@ -195,14 +202,14 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     LabelsToNodesInfo ltni = response.getEntity(LabelsToNodesInfo.class);
     assertEquals(2, ltni.getLabelsToNodes().size());
-    NodeIDsInfo nodes = ltni.getLabelsToNodes().get("b");
+    NodeIDsInfo nodes = ltni.getLabelsToNodes().get(new NodeLabelInfo("b"));
     assertTrue(nodes.getNodeIDs().contains("nid2:0"));
     assertTrue(nodes.getNodeIDs().contains("nid1:0"));
-    nodes = ltni.getLabelsToNodes().get("a");
+    nodes = ltni.getLabelsToNodes().get(new NodeLabelInfo("a"));
     assertTrue(nodes.getNodeIDs().contains("nid:0"));
 
-    // Verify, using get-labels-to-Nodes for specifiedset of labels
-    MultivaluedMapImpl params = new MultivaluedMapImpl();
+    // Verify, using get-labels-to-Nodes for specified set of labels
+    params = new MultivaluedMapImpl();
     params.add("labels", "a");
     response =
         r.path("ws").path("v1").path("cluster")
@@ -213,7 +220,7 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     ltni = response.getEntity(LabelsToNodesInfo.class);
     assertEquals(1, ltni.getLabelsToNodes().size());
-    nodes = ltni.getLabelsToNodes().get("a");
+    nodes = ltni.getLabelsToNodes().get(new NodeLabelInfo("a"));
     assertTrue(nodes.getNodeIDs().contains("nid:0"));
 
     // Verify
@@ -223,18 +230,20 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("get-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    json = response.getEntity(JSONObject.class);
-    assertEquals("a", json.getString("nodeLabels"));
+    nlsifo = response.getEntity(NodeLabelsInfo.class);
+    assertTrue(nlsifo.getNodeLabelsName().contains("a"));
 
     
     // Replace
+    params = new MultivaluedMapImpl();
+    params.add("labels", "b");
     response =
         r.path("ws").path("v1").path("cluster")
             .path("nodes").path("nid:0")
             .path("replace-labels")
             .queryParam("user.name", userName)
+            .queryParams(params)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\":\"b\"}", MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
     LOG.info("posted node nodelabel");
 
@@ -245,20 +254,21 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("get-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    json = response.getEntity(JSONObject.class);
-    assertEquals("b", json.getString("nodeLabels"));
+    nlsifo = response.getEntity(NodeLabelsInfo.class);
+    assertTrue(nlsifo.getNodeLabelsName().contains("b"));
             
     // Replace labels using node-to-labels
-    NodeToLabelsInfo ntli = new NodeToLabelsInfo();
-    NodeLabelsInfo nli = new NodeLabelsInfo();
-    nli.getNodeLabels().add("a");
-    ntli.getNodeToLabels().put("nid:0", nli);
+    NodeToLabelsEntryList ntli = new NodeToLabelsEntryList();
+    ArrayList<String> labels = new ArrayList<String>();
+    labels.add("a");
+    NodeToLabelsEntry nli = new NodeToLabelsEntry("nid:0", labels);
+    ntli.getNodeToLabels().add(nli);
     response =
         r.path("ws").path("v1").path("cluster")
             .path("replace-node-to-labels")
             .queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(ntli, NodeToLabelsInfo.class),
+            .entity(toJson(ntli, NodeToLabelsEntryList.class),
               MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
         
@@ -268,19 +278,21 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("get-node-to-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    ntli = response.getEntity(NodeToLabelsInfo.class);
-    nli = ntli.getNodeToLabels().get("nid:0");
-    assertEquals(1, nli.getNodeLabels().size());
-    assertTrue(nli.getNodeLabels().contains("a"));
+    NodeToLabelsInfo ntlinfo = response.getEntity(NodeToLabelsInfo.class);
+    NodeLabelsInfo nlinfo = ntlinfo.getNodeToLabels().get("nid:0");
+    assertEquals(1, nlinfo.getNodeLabels().size());
+    assertTrue(nlinfo.getNodeLabelsName().contains("a"));
     
     // Remove all
+    params = new MultivaluedMapImpl();
+    params.add("labels", "");
     response =
         r.path("ws").path("v1").path("cluster")
             .path("nodes").path("nid:0")
             .path("replace-labels")
             .queryParam("user.name", userName)
+            .queryParams(params)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\"}", MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
     LOG.info("posted node nodelabel");
     // Verify
@@ -290,18 +302,19 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("get-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    json = response.getEntity(JSONObject.class);
-    assertEquals("", json.getString("nodeLabels"));
+    nlsifo = response.getEntity(NodeLabelsInfo.class);
+    assertTrue(nlsifo.getNodeLabelsName().contains(""));
     
     // Add a label back for auth tests
+    params = new MultivaluedMapImpl();
+    params.add("labels", "a");
     response =
         r.path("ws").path("v1").path("cluster")
             .path("nodes").path("nid:0")
             .path("replace-labels")
             .queryParam("user.name", userName)
+            .queryParams(params)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\": \"a\"}",
-              MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
     LOG.info("posted node nodelabel");
 
@@ -312,18 +325,19 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("get-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    json = response.getEntity(JSONObject.class);
-    assertEquals("a", json.getString("nodeLabels"));
+    nlsifo = response.getEntity(NodeLabelsInfo.class);
+    assertTrue(nlsifo.getNodeLabelsName().contains("a"));
     
     // Auth fail replace labels on node
+    params = new MultivaluedMapImpl();
+    params.add("labels", "b");
     response =
         r.path("ws").path("v1").path("cluster")
             .path("nodes").path("nid:0")
             .path("replace-labels")
             .queryParam("user.name", notUserName)
+            .queryParams(params)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\": [\"b\"]}",
-              MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
     // Verify
     response =
@@ -332,8 +346,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("get-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    json = response.getEntity(JSONObject.class);
-    assertEquals("a", json.getString("nodeLabels"));
+    nlsifo = response.getEntity(NodeLabelsInfo.class);
+    assertTrue(nlsifo.getNodeLabelsName().contains("a"));
     
     // Fail to add a label with post
     response =
@@ -349,17 +363,18 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("get-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    json = response.getEntity(JSONObject.class);
-    jarr = json.getJSONArray("nodeLabels");
-    assertEquals(2, jarr.length());
+    nlsifo = response.getEntity(NodeLabelsInfo.class);
+    assertEquals(2, nlsifo.getNodeLabels().size());
     
     // Remove cluster label (succeed, we no longer need it)
+    params = new MultivaluedMapImpl();
+    params.add("labels", "b");
     response =
         r.path("ws").path("v1").path("cluster")
             .path("remove-node-labels")
             .queryParam("user.name", userName)
+            .queryParams(params)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\":\"b\"}", MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
     // Verify
     response =
@@ -367,17 +382,19 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("get-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    json = response.getEntity(JSONObject.class);
-    assertEquals("a", json.getString("nodeLabels"));
-    
+    nlsifo = response.getEntity(NodeLabelsInfo.class);
+    assertEquals("a", nlsifo.getNodeLabelsInfo().get(0).getName());
+    assertEquals(1, nlsifo.getNodeLabels().size());
     
     // Remove cluster label with post
+    params = new MultivaluedMapImpl();
+    params.add("labels", "a");
     response =
         r.path("ws").path("v1").path("cluster")
             .path("remove-node-labels")
             .queryParam("user.name", userName)
+            .queryParams(params)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\":\"a\"}", MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
     // Verify
     response =
@@ -385,12 +402,15 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("get-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    String res = response.getEntity(String.class);
-    assertTrue(res.equals("null"));
+    nlsifo = response.getEntity(NodeLabelsInfo.class);
+    assertEquals(0, nlsifo.getNodeLabels().size());
 
     // Following test cases are to test replace when distributed node label
     // configuration is on
     // Reset for testing : add cluster labels
+    nlsifo = new NodeLabelsInfo();
+    nlsifo.getNodeLabelsInfo().add(new NodeLabelInfo("x"));
+    nlsifo.getNodeLabelsInfo().add(new NodeLabelInfo("y"));
     response =
         r.path("ws")
             .path("v1")
@@ -398,14 +418,18 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("add-node-labels")
             .queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\":[\"x\",\"y\"]}",
+            .entity(toJson(nlsifo, NodeLabelsInfo.class),
                 MediaType.APPLICATION_JSON).post(ClientResponse.class);
     // Reset for testing : Add labels to a node
+    nlsifo = new NodeLabelsInfo();
+    nlsifo.getNodeLabelsInfo().add(new NodeLabelInfo("y"));
+    params = new MultivaluedMapImpl();
+    params.add("labels", "y");
     response =
         r.path("ws").path("v1").path("cluster").path("nodes").path("nid:0")
             .path("replace-labels").queryParam("user.name", userName)
+            .queryParams(params)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\": [\"y\"]}", MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
     LOG.info("posted node nodelabel");
 
@@ -413,10 +437,11 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
     rmWebService.isDistributedNodeLabelConfiguration = true;
 
     // Case1 : Replace labels using node-to-labels
-    ntli = new NodeToLabelsInfo();
-    nli = new NodeLabelsInfo();
-    nli.getNodeLabels().add("x");
-    ntli.getNodeToLabels().put("nid:0", nli);
+    ntli = new NodeToLabelsEntryList();
+    labels = new ArrayList<String>();
+    labels.add("x");
+    nli = new NodeToLabelsEntry("nid:0", labels);
+    ntli.getNodeToLabels().add(nli);
     response =
         r.path("ws")
             .path("v1")
@@ -424,7 +449,7 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("replace-node-to-labels")
             .queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(ntli, NodeToLabelsInfo.class),
+            .entity(toJson(ntli, NodeToLabelsEntryList.class),
                 MediaType.APPLICATION_JSON).post(ClientResponse.class);
 
     // Verify, using node-to-labels that previous operation has failed
@@ -433,17 +458,17 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    ntli = response.getEntity(NodeToLabelsInfo.class);
-    nli = ntli.getNodeToLabels().get("nid:0");
-    assertEquals(1, nli.getNodeLabels().size());
-    assertFalse(nli.getNodeLabels().contains("x"));
+    ntlinfo = response.getEntity(NodeToLabelsInfo.class);
+    nlinfo = ntlinfo.getNodeToLabels().get("nid:0");
+    assertEquals(1, nlinfo.getNodeLabels().size());
+    assertFalse(nlinfo.getNodeLabels().contains("x"));
 
     // Case2 : failure to Replace labels using replace-labels
     response =
         r.path("ws").path("v1").path("cluster").path("nodes").path("nid:0")
             .path("replace-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\": [\"x\"]}", MediaType.APPLICATION_JSON)
+            .entity("{\"nodeLabelName\": [\"x\"]}", MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
     LOG.info("posted node nodelabel");
 
@@ -453,18 +478,20 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    ntli = response.getEntity(NodeToLabelsInfo.class);
-    nli = ntli.getNodeToLabels().get("nid:0");
-    assertEquals(1, nli.getNodeLabels().size());
-    assertFalse(nli.getNodeLabels().contains("x"));
+    ntlinfo = response.getEntity(NodeToLabelsInfo.class);
+    nlinfo = ntlinfo.getNodeToLabels().get("nid:0");
+    assertEquals(1, nlinfo.getNodeLabels().size());
+    assertFalse(nlinfo.getNodeLabels().contains("x"));
 
-    //  Case3 : Remove cluster label should be successfull
+    //  Case3 : Remove cluster label should be successful
+    params = new MultivaluedMapImpl();
+    params.add("labels", "x");
     response =
         r.path("ws").path("v1").path("cluster")
             .path("remove-node-labels")
             .queryParam("user.name", userName)
+            .queryParams(params)
             .accept(MediaType.APPLICATION_JSON)
-            .entity("{\"nodeLabels\":\"x\"}", MediaType.APPLICATION_JSON)
             .post(ClientResponse.class);
     // Verify
     response =
@@ -472,8 +499,52 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase {
             .path("get-node-labels").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
-    json = response.getEntity(JSONObject.class);
-    assertEquals("y", json.getString("nodeLabels"));
+    nlsifo = response.getEntity(NodeLabelsInfo.class);
+    assertEquals("y", nlsifo.getNodeLabelsInfo().get(0).getName());
+
+    // Remove y
+    params = new MultivaluedMapImpl();
+    params.add("labels", "y");
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("remove-node-labels")
+            .queryParam("user.name", userName)
+            .queryParams(params)
+            .accept(MediaType.APPLICATION_JSON)
+            .post(ClientResponse.class);
+
+    // Verify
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("get-node-labels").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    nlsifo = response.getEntity(NodeLabelsInfo.class);
+    assertTrue(nlsifo.getNodeLabelsInfo().isEmpty());
+
+    // add a new nodelabel with exclusity
+    nlsifo = new NodeLabelsInfo();
+    nlsifo.getNodeLabelsInfo().add(new NodeLabelInfo("z", false));
+    response =
+        r.path("ws")
+            .path("v1")
+            .path("cluster")
+            .path("add-node-labels")
+            .queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity(toJson(nlsifo, NodeLabelsInfo.class),
+                MediaType.APPLICATION_JSON).post(ClientResponse.class);
+
+    // Verify
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("get-node-labels").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    nlsifo = response.getEntity(NodeLabelsInfo.class);
+    assertEquals("z", nlsifo.getNodeLabelsInfo().get(0).getName());
+    assertEquals(false, nlsifo.getNodeLabelsInfo().get(0).getExclusivity());
+    assertEquals(1, nlsifo.getNodeLabels().size());
   }
 
   @SuppressWarnings("rawtypes")


[11/37] hadoop git commit: MAPREDUCE-6251. Added a new config for JobClient to retry JobStatus calls so that they don't fail on history-server backed by DFSes with not so strong guarantees. Contributed by Craig Welch.

Posted by ji...@apache.org.
MAPREDUCE-6251. Added a new config for JobClient to retry JobStatus calls so that they don't fail on history-server backed by DFSes with not so strong guarantees. Contributed by Craig Welch.


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

Branch: refs/heads/HDFS-7240
Commit: f24452d14e9ba48cdb82e5e6e5c10ce5b1407308
Parents: fe0df59
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Tue May 12 12:11:42 2015 -0700
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Tue May 12 12:11:42 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  5 ++
 .../org/apache/hadoop/mapred/JobClient.java     | 51 +++++++++++----
 .../apache/hadoop/mapreduce/MRJobConfig.java    | 15 +++++
 .../src/main/resources/mapred-default.xml       | 17 +++++
 .../apache/hadoop/mapred/JobClientUnitTest.java | 65 ++++++++++++++++++++
 5 files changed, 142 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24452d1/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 15cdf90..fc98376 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -328,6 +328,7 @@ Release 2.8.0 - UNRELEASED
   OPTIMIZATIONS
 
   BUG FIXES
+
     MAPREDUCE-6314. TestPipeApplication fails on trunk.
     (Varun Vasudev via harsh)
 
@@ -450,6 +451,10 @@ Release 2.7.1 - UNRELEASED
     MAPREDUCE-6259. IllegalArgumentException due to missing job submit time
     (zhihai xu via jlowe)
 
+    MAPREDUCE-6251. Added a new config for JobClient to retry JobStatus calls so
+    that they don't fail on history-server backed by DFSes with not so strong
+    guarantees. (Craig Welch via vinodkv)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24452d1/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java
index e91fbfe..cf123c7 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.mapred.ClusterStatus.BlackListInfo;
 import org.apache.hadoop.mapreduce.Cluster;
 import org.apache.hadoop.mapreduce.ClusterMetrics;
 import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.QueueInfo;
 import org.apache.hadoop.mapreduce.TaskTrackerInfo;
 import org.apache.hadoop.mapreduce.TaskType;
@@ -154,6 +155,10 @@ public class JobClient extends CLI {
   public static enum TaskStatusFilter { NONE, KILLED, FAILED, SUCCEEDED, ALL }
   private TaskStatusFilter taskOutputFilter = TaskStatusFilter.FAILED; 
   
+  private int maxRetry = MRJobConfig.DEFAULT_MR_CLIENT_JOB_MAX_RETRIES;
+  private long retryInterval =
+      MRJobConfig.DEFAULT_MR_CLIENT_JOB_RETRY_INTERVAL;
+
   static{
     ConfigUtil.loadResources();
   }
@@ -469,6 +474,14 @@ public class JobClient extends CLI {
     setConf(conf);
     cluster = new Cluster(conf);
     clientUgi = UserGroupInformation.getCurrentUser();
+
+    maxRetry = conf.getInt(MRJobConfig.MR_CLIENT_JOB_MAX_RETRIES,
+      MRJobConfig.DEFAULT_MR_CLIENT_JOB_MAX_RETRIES);
+
+    retryInterval =
+      conf.getLong(MRJobConfig.MR_CLIENT_JOB_RETRY_INTERVAL,
+        MRJobConfig.DEFAULT_MR_CLIENT_JOB_RETRY_INTERVAL);
+
   }
 
   /**
@@ -581,16 +594,8 @@ public class JobClient extends CLI {
       }
     });
   }
-  /**
-   * Get an {@link RunningJob} object to track an ongoing job.  Returns
-   * null if the id does not correspond to any known job.
-   * 
-   * @param jobid the jobid of the job.
-   * @return the {@link RunningJob} handle to track the job, null if the 
-   *         <code>jobid</code> doesn't correspond to any known job.
-   * @throws IOException
-   */
-  public RunningJob getJob(final JobID jobid) throws IOException {
+
+  protected RunningJob getJobInner(final JobID jobid) throws IOException {
     try {
       
       Job job = getJobUsingCluster(jobid);
@@ -607,7 +612,31 @@ public class JobClient extends CLI {
     return null;
   }
 
-  /**@deprecated Applications should rather use {@link #getJob(JobID)}. 
+  /**
+   * Get an {@link RunningJob} object to track an ongoing job.  Returns
+   * null if the id does not correspond to any known job.
+   *
+   * @param jobid the jobid of the job.
+   * @return the {@link RunningJob} handle to track the job, null if the
+   *         <code>jobid</code> doesn't correspond to any known job.
+   * @throws IOException
+   */
+  public RunningJob getJob(final JobID jobid) throws IOException {
+     for (int i = 0;i <= maxRetry;i++) {
+       if (i > 0) {
+         try {
+           Thread.sleep(retryInterval);
+         } catch (Exception e) { }
+       }
+       RunningJob job = getJobInner(jobid);
+       if (job != null) {
+         return job;
+       }
+     }
+     return null;
+  }
+
+  /**@deprecated Applications should rather use {@link #getJob(JobID)}.
    */
   @Deprecated
   public RunningJob getJob(String jobid) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24452d1/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index 49345cd..fb4064c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -469,6 +469,21 @@ public interface MRJobConfig {
     MR_PREFIX + "client.max-retries";
   public static final int DEFAULT_MR_CLIENT_MAX_RETRIES = 3;
   
+  /**
+   * How many times to retry jobclient calls (via getjob)
+   */
+  public static final String MR_CLIENT_JOB_MAX_RETRIES =
+      MR_PREFIX + "client.job.max-retries";
+  public static final int DEFAULT_MR_CLIENT_JOB_MAX_RETRIES = 0;
+
+  /**
+   * How long to wait between jobclient retries on failure
+   */
+  public static final String MR_CLIENT_JOB_RETRY_INTERVAL =
+      MR_PREFIX + "client.job.retry-interval";
+  public static final long DEFAULT_MR_CLIENT_JOB_RETRY_INTERVAL =
+      2000;
+
   /** The staging directory for map reduce.*/
   public static final String MR_AM_STAGING_DIR = 
     MR_AM_PREFIX+"staging-dir";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24452d1/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 a5e76b3..5daf66d 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
@@ -1391,6 +1391,23 @@
 </property>
 
 <property>
+  <name>yarn.app.mapreduce.client.job.max-retries</name>
+  <value>0</value>
+  <description>The number of retries the client will make for getJob and
+  dependent calls.  The default is 0 as this is generally only needed for
+  non-HDFS DFS where additional, high level retries are required to avoid
+  spurious failures during the getJob call.  30 is a good value for
+  WASB</description>
+</property>
+
+<property>
+  <name>yarn.app.mapreduce.client.job.retry-interval</name>
+  <value>2000</value>
+  <description>The delay between getJob retries in ms for retries configured
+  with yarn.app.mapreduce.client.job.max-retries.</description>
+</property>
+
+<property>
   <description>CLASSPATH for MR applications. A comma-separated list
   of CLASSPATH entries. If mapreduce.application.framework is set then this
   must specify the appropriate classpath for that archive, and the name of

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f24452d1/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/JobClientUnitTest.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/JobClientUnitTest.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/JobClientUnitTest.java
index 8dfac89..84b76bf 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/JobClientUnitTest.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/JobClientUnitTest.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.mapred;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
 import static org.mockito.Matchers.isA;
 import static org.mockito.Mockito.atLeastOnce;
 import static org.mockito.Mockito.mock;
@@ -35,6 +36,7 @@ import org.apache.hadoop.mapreduce.Cluster;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobPriority;
 import org.apache.hadoop.mapreduce.JobStatus;
+import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TaskReport;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.junit.Assert;
@@ -52,6 +54,42 @@ public class JobClientUnitTest {
     void setCluster(Cluster cluster) {
       this.cluster = cluster;
     }
+
+  }
+
+  public class TestJobClientGetJob extends TestJobClient {
+
+    int lastGetJobRetriesCounter = 0;
+    int getJobRetriesCounter = 0;
+    int getJobRetries = 0;
+    RunningJob runningJob;
+
+    TestJobClientGetJob(JobConf jobConf) throws IOException {
+      super(jobConf);
+    }
+
+    public int getLastGetJobRetriesCounter() {
+      return lastGetJobRetriesCounter;
+    }
+
+    public void setGetJobRetries(int getJobRetries) {
+      this.getJobRetries = getJobRetries;
+    }
+
+    public void setRunningJob(RunningJob runningJob) {
+      this.runningJob = runningJob;
+    }
+
+    protected RunningJob getJobInner(final JobID jobid) throws IOException {
+      if (getJobRetriesCounter >= getJobRetries) {
+        lastGetJobRetriesCounter = getJobRetriesCounter;
+        getJobRetriesCounter = 0;
+        return runningJob;
+      }
+      getJobRetriesCounter++;
+      return null;
+    }
+
   }
 
   @Test
@@ -124,6 +162,7 @@ public class JobClientUnitTest {
 
     JobStatus mockJobStatus = mock(JobStatus.class);
     when(mockJobStatus.getJobID()).thenReturn(jobID);
+    when(mockJobStatus.getJobName()).thenReturn(jobID.toString());
     when(mockJobStatus.getState()).thenReturn(JobStatus.State.RUNNING);
     when(mockJobStatus.getStartTime()).thenReturn(startTime);
     when(mockJobStatus.getUsername()).thenReturn("mockuser");
@@ -181,4 +220,30 @@ public class JobClientUnitTest {
     assertNull(client.getJob(id));
   }
 
+  @Test
+  public void testGetJobRetry() throws Exception {
+
+    //To prevent the test from running for a very long time, lower the retry
+    JobConf conf = new JobConf();
+    conf.set(MRJobConfig.MR_CLIENT_JOB_MAX_RETRIES, "3");
+
+    TestJobClientGetJob client = new TestJobClientGetJob(conf);
+    JobID id = new JobID("ajob",1);
+    RunningJob rj = mock(RunningJob.class);
+    client.setRunningJob(rj);
+
+    //no retry
+    assertNotNull(client.getJob(id));
+    assertEquals(client.getLastGetJobRetriesCounter(), 0);
+
+    //3 retry
+    client.setGetJobRetries(3);
+    assertNotNull(client.getJob(id));
+    assertEquals(client.getLastGetJobRetriesCounter(), 3);
+
+    //beyond MAPREDUCE_JOBCLIENT_GETJOB_MAX_RETRY_KEY, will get null
+    client.setGetJobRetries(5);
+    assertNull(client.getJob(id));
+  }
+
 }


[25/37] hadoop git commit: Update fix version for YARN-3457 and YARN-3537.

Posted by ji...@apache.org.
Update fix version for YARN-3457 and YARN-3537.


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

Branch: refs/heads/HDFS-7240
Commit: f7de6198da030cb9cd62dcfd3c378dbbf857e017
Parents: 7f19e7a
Author: Jason Lowe <jl...@apache.org>
Authored: Wed May 13 20:52:17 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed May 13 20:53:16 2015 +0000

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7de6198/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 38f5e81..146690b 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -291,9 +291,6 @@ Release 2.8.0 - UNRELEASED
 
     YARN-3110. Few issues in ApplicationHistory web ui. (Naganarasimha G R via xgong)
 
-    YARN-3457. NPE when NodeManager.serviceInit fails and stopRecoveryStore called.
-    (Bibin A Chundatt via ozawa)
-
     YARN-3459. Fix failiure of TestLog4jWarningErrorMetricsAppender.
     (Varun Vasudev via wangda)
 
@@ -320,9 +317,6 @@ Release 2.8.0 - UNRELEASED
 
     YARN-3444. Fix typo capabililty. (Gabor Liptak via aajisaka)
 
-    YARN-3537. NPE when NodeManager.serviceInit fails and stopRecoveryStore
-    invoked (Brahma Reddy Battula via jlowe)
-
     YARN-3530. ATS throws exception on trying to filter results without otherinfo.
     (zhijie shen via xgong)
 
@@ -488,6 +482,12 @@ Release 2.7.1 - UNRELEASED
     YARN-3626. On Windows localized resources are not moved to the front
     of the classpath when they should be. (Craig Welch via xgong)
 
+    YARN-3457. NPE when NodeManager.serviceInit fails and stopRecoveryStore called.
+    (Bibin A Chundatt via ozawa)
+
+    YARN-3537. NPE when NodeManager.serviceInit fails and stopRecoveryStore
+    invoked (Brahma Reddy Battula via jlowe)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES


[04/37] hadoop git commit: MAPREDUCE-6360. TestMapreduceConfigFields is placed in wrong dir, introducing compile error (Contributed by Arshad Mohammad)

Posted by ji...@apache.org.
MAPREDUCE-6360. TestMapreduceConfigFields is placed in wrong dir, introducing compile error (Contributed by Arshad Mohammad)


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

Branch: refs/heads/HDFS-7240
Commit: 360dff5903085c3c7f02ccf9f17d71a842275e48
Parents: 987abc9
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue May 12 12:49:16 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Tue May 12 12:49:16 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 +
 .../mapred/TestMapreduceConfigFields.java       | 76 --------------------
 .../mapreduce/TestMapreduceConfigFields.java    | 76 ++++++++++++++++++++
 3 files changed, 79 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/360dff59/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index f48f847..ca92a97 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -411,6 +411,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-5465. Tasks are often killed before they exit on their own
     (Ming Ma via jlowe)
 
+    MAPREDUCE-6360. TestMapreduceConfigFields is placed in wrong dir, 
+    introducing compile error (Arshad Mohammad via vinayakumarb)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/360dff59/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestMapreduceConfigFields.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestMapreduceConfigFields.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestMapreduceConfigFields.java
deleted file mode 100644
index 7f18714..0000000
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestMapreduceConfigFields.java
+++ /dev/null
@@ -1,76 +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.mapreduce;
-
-import java.util.HashSet;
-
-import org.apache.hadoop.conf.TestConfigurationFieldsBase;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.ShuffleHandler;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
-
-/**
- * Unit test class to compare the following MR Configuration classes:
- * <p></p>
- * {@link org.apache.hadoop.mapreduce.MRJobConfig}
- * {@link org.apache.hadoop.mapreduce.MRConfig}
- * {@link org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig}
- * {@link org.apache.hadoop.mapred.ShuffleHandler}
- * {@link org.apache.hadoop.mapreduce.lib.output.FileOutputFormat}
- * {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat}
- * {@link org.apache.hadoop.mapreduce.Job}
- * {@link org.apache.hadoop.mapreduce.lib.input.NLineInputFormat}
- * {@link org.apache.hadoop.mapred.JobConf}
- * <p></p>
- * against mapred-default.xml for missing properties.  Currently only
- * throws an error if the class is missing a property.
- * <p></p>
- * Refer to {@link org.apache.hadoop.conf.TestConfigurationFieldsBase}
- * for how this class works.
- */
-public class TestMapreduceConfigFields extends TestConfigurationFieldsBase {
-
-  @SuppressWarnings("deprecation")
-  @Override
-  public void initializeMemberVariables() {
-    xmlFilename = new String("mapred-default.xml");
-    configurationClasses = new Class[] { MRJobConfig.class, MRConfig.class,
-        JHAdminConfig.class, ShuffleHandler.class, FileOutputFormat.class,
-	FileInputFormat.class, Job.class, NLineInputFormat.class,
-	JobConf.class, FileOutputCommitter.class };
-
-    // Initialize used variables
-    configurationPropsToSkipCompare = new HashSet<String>();
-
-    // Set error modes
-    errorIfMissingConfigProps = true;
-    errorIfMissingXmlProps = false;
-
-    // Ignore deprecated MR1 properties in JobConf
-    configurationPropsToSkipCompare
-            .add(JobConf.MAPRED_JOB_MAP_MEMORY_MB_PROPERTY);
-    configurationPropsToSkipCompare
-            .add(JobConf.MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/360dff59/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/TestMapreduceConfigFields.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/TestMapreduceConfigFields.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/TestMapreduceConfigFields.java
new file mode 100644
index 0000000..7f18714
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/TestMapreduceConfigFields.java
@@ -0,0 +1,76 @@
+/**
+ * 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.mapreduce;
+
+import java.util.HashSet;
+
+import org.apache.hadoop.conf.TestConfigurationFieldsBase;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.ShuffleHandler;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
+
+/**
+ * Unit test class to compare the following MR Configuration classes:
+ * <p></p>
+ * {@link org.apache.hadoop.mapreduce.MRJobConfig}
+ * {@link org.apache.hadoop.mapreduce.MRConfig}
+ * {@link org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig}
+ * {@link org.apache.hadoop.mapred.ShuffleHandler}
+ * {@link org.apache.hadoop.mapreduce.lib.output.FileOutputFormat}
+ * {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat}
+ * {@link org.apache.hadoop.mapreduce.Job}
+ * {@link org.apache.hadoop.mapreduce.lib.input.NLineInputFormat}
+ * {@link org.apache.hadoop.mapred.JobConf}
+ * <p></p>
+ * against mapred-default.xml for missing properties.  Currently only
+ * throws an error if the class is missing a property.
+ * <p></p>
+ * Refer to {@link org.apache.hadoop.conf.TestConfigurationFieldsBase}
+ * for how this class works.
+ */
+public class TestMapreduceConfigFields extends TestConfigurationFieldsBase {
+
+  @SuppressWarnings("deprecation")
+  @Override
+  public void initializeMemberVariables() {
+    xmlFilename = new String("mapred-default.xml");
+    configurationClasses = new Class[] { MRJobConfig.class, MRConfig.class,
+        JHAdminConfig.class, ShuffleHandler.class, FileOutputFormat.class,
+	FileInputFormat.class, Job.class, NLineInputFormat.class,
+	JobConf.class, FileOutputCommitter.class };
+
+    // Initialize used variables
+    configurationPropsToSkipCompare = new HashSet<String>();
+
+    // Set error modes
+    errorIfMissingConfigProps = true;
+    errorIfMissingXmlProps = false;
+
+    // Ignore deprecated MR1 properties in JobConf
+    configurationPropsToSkipCompare
+            .add(JobConf.MAPRED_JOB_MAP_MEMORY_MB_PROPERTY);
+    configurationPropsToSkipCompare
+            .add(JobConf.MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY);
+  }
+
+}


[31/37] hadoop git commit: HDFS-7728. Avoid updating quota usage while loading edits. Contributed by Jing Zhao.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
index 0788e75..62aaccd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
@@ -45,21 +45,23 @@ public class FileDiffList extends
 
   public void destroyAndCollectSnapshotBlocks(
       BlocksMapUpdateInfo collectedBlocks) {
-    for(FileDiff d : asList())
+    for (FileDiff d : asList()) {
       d.destroyAndCollectSnapshotBlocks(collectedBlocks);
+    }
   }
 
   public void saveSelf2Snapshot(int latestSnapshotId, INodeFile iNodeFile,
       INodeFileAttributes snapshotCopy, boolean withBlocks) {
     final FileDiff diff =
         super.saveSelf2Snapshot(latestSnapshotId, iNodeFile, snapshotCopy);
-    if(withBlocks)  // Store blocks if this is the first update
+    if (withBlocks) {  // Store blocks if this is the first update
       diff.setBlocks(iNodeFile.getBlocks());
+    }
   }
 
   public BlockInfoContiguous[] findEarlierSnapshotBlocks(int snapshotId) {
     assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
-    if(snapshotId == Snapshot.CURRENT_STATE_ID) {
+    if (snapshotId == Snapshot.CURRENT_STATE_ID) {
       return null;
     }
     List<FileDiff> diffs = this.asList();
@@ -76,15 +78,15 @@ public class FileDiffList extends
 
   public BlockInfoContiguous[] findLaterSnapshotBlocks(int snapshotId) {
     assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
-    if(snapshotId == Snapshot.CURRENT_STATE_ID) {
+    if (snapshotId == Snapshot.CURRENT_STATE_ID) {
       return null;
     }
     List<FileDiff> diffs = this.asList();
     int i = Collections.binarySearch(diffs, snapshotId);
     BlockInfoContiguous[] blocks = null;
-    for(i = i >= 0 ? i+1 : -i-1; i < diffs.size(); i++) {
+    for (i = i >= 0 ? i+1 : -i-1; i < diffs.size(); i++) {
       blocks = diffs.get(i).getBlocks();
-      if(blocks != null) {
+      if (blocks != null) {
         break;
       }
     }
@@ -99,7 +101,7 @@ public class FileDiffList extends
   void combineAndCollectSnapshotBlocks(
       INode.ReclaimContext reclaimContext, INodeFile file, FileDiff removed) {
     BlockInfoContiguous[] removedBlocks = removed.getBlocks();
-    if(removedBlocks == null) {
+    if (removedBlocks == null) {
       FileWithSnapshotFeature sf = file.getFileWithSnapshotFeature();
       assert sf != null : "FileWithSnapshotFeature is null";
       if(sf.isCurrentFileDeleted())
@@ -109,8 +111,9 @@ public class FileDiffList extends
     int p = getPrior(removed.getSnapshotId(), true);
     FileDiff earlierDiff = p == Snapshot.NO_SNAPSHOT_ID ? null : getDiffById(p);
     // Copy blocks to the previous snapshot if not set already
-    if(earlierDiff != null)
+    if (earlierDiff != null) {
       earlierDiff.setBlocks(removedBlocks);
+    }
     BlockInfoContiguous[] earlierBlocks =
         (earlierDiff == null ? new BlockInfoContiguous[]{} : earlierDiff.getBlocks());
     // Find later snapshot (or file itself) with blocks
@@ -128,13 +131,13 @@ public class FileDiffList extends
     // Check if last block is part of truncate recovery
     BlockInfoContiguous lastBlock = file.getLastBlock();
     Block dontRemoveBlock = null;
-    if(lastBlock != null && lastBlock.getBlockUCState().equals(
+    if (lastBlock != null && lastBlock.getBlockUCState().equals(
         HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) {
       dontRemoveBlock = ((BlockInfoContiguousUnderConstruction) lastBlock)
           .getTruncateBlock();
     }
     // Collect the remaining blocks of the file, ignoring truncate block
-    for(;i < removedBlocks.length; i++) {
+    for (;i < removedBlocks.length; i++) {
       if(dontRemoveBlock == null || !removedBlocks[i].equals(dontRemoveBlock)) {
         reclaimContext.collectedBlocks().addDeleteBlock(removedBlocks[i]);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
index 213c186..555a662 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
@@ -116,21 +116,25 @@ public class FileWithSnapshotFeature implements INode.Feature {
     return (isCurrentFileDeleted()? "(DELETED), ": ", ") + diffs;
   }
   
-  public QuotaCounts cleanFile(INode.ReclaimContext reclaimContext,
-      final INodeFile file, final int snapshotId,
-      int priorSnapshotId) {
+  public void cleanFile(INode.ReclaimContext reclaimContext,
+      final INodeFile file, final int snapshotId, int priorSnapshotId,
+      byte storagePolicyId) {
     if (snapshotId == Snapshot.CURRENT_STATE_ID) {
       // delete the current file while the file has snapshot feature
       if (!isCurrentFileDeleted()) {
         file.recordModification(priorSnapshotId);
         deleteCurrentFile();
       }
+      final BlockStoragePolicy policy = reclaimContext.storagePolicySuite()
+          .getPolicy(storagePolicyId);
+      QuotaCounts old = file.storagespaceConsumed(policy);
       collectBlocksAndClear(reclaimContext, file);
-      return new QuotaCounts.Builder().build();
+      QuotaCounts current = file.storagespaceConsumed(policy);
+      reclaimContext.quotaDelta().add(old.subtract(current));
     } else { // delete the snapshot
       priorSnapshotId = getDiffs().updatePrior(snapshotId, priorSnapshotId);
-      return diffs.deleteSnapshotDiff(reclaimContext,
-          snapshotId, priorSnapshotId, file);
+      diffs.deleteSnapshotDiff(reclaimContext, snapshotId, priorSnapshotId,
+          file);
     }
   }
   
@@ -138,8 +142,8 @@ public class FileWithSnapshotFeature implements INode.Feature {
     this.diffs.clear();
   }
   
-  public QuotaCounts updateQuotaAndCollectBlocks(
-      INode.ReclaimContext reclaimContext, INodeFile file, FileDiff removed) {
+  public void updateQuotaAndCollectBlocks(INode.ReclaimContext reclaimContext,
+      INodeFile file, FileDiff removed) {
     byte storagePolicyID = file.getStoragePolicyID();
     BlockStoragePolicy bsp = null;
     if (storagePolicyID != HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
@@ -179,8 +183,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
     getDiffs().combineAndCollectSnapshotBlocks(reclaimContext, file, removed);
 
     QuotaCounts current = file.storagespaceConsumed(bsp);
-    oldCounts.subtract(current);
-    return oldCounts;
+    reclaimContext.quotaDelta().add(oldCounts.subtract(current));
   }
 
   /**
@@ -191,7 +194,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
       INode.ReclaimContext reclaimContext, final INodeFile file) {
     // check if everything is deleted.
     if (isCurrentFileDeleted() && getDiffs().asList().isEmpty()) {
-      file.destroyAndCollectBlocks(reclaimContext);
+      file.clearFile(reclaimContext);
       return;
     }
     // find max file size.
@@ -199,7 +202,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
     FileDiff diff = getDiffs().getLast();
     if (isCurrentFileDeleted()) {
       max = diff == null? 0: diff.getFileSize();
-    } else { 
+    } else {
       max = file.computeFileSize();
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
index bc9544b..c738d64 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
 import org.apache.hadoop.metrics2.util.MBeans;
@@ -211,7 +210,6 @@ public class SnapshotManager implements SnapshotStatsMXBean {
       // We have reached the maximum allowable snapshot ID and since we don't
       // handle rollover we will fail all subsequent snapshot creation
       // requests.
-      //
       throw new SnapshotException(
           "Failed to create the snapshot. The FileSystem has run out of " +
           "snapshot IDs and ID rollover is not supported.");
@@ -228,17 +226,13 @@ public class SnapshotManager implements SnapshotStatsMXBean {
   /**
    * Delete a snapshot for a snapshottable directory
    * @param snapshotName Name of the snapshot to be deleted
-   * @param collectedBlocks Used to collect information to update blocksMap
-   * @throws IOException
+   * @param reclaimContext Used to collect information to reclaim blocks
+   *                       and inodes
    */
   public void deleteSnapshot(final INodesInPath iip, final String snapshotName,
-      BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
-      throws IOException {
+      INode.ReclaimContext reclaimContext) throws IOException {
     INodeDirectory srcRoot = getSnapshottableRoot(iip);
-    srcRoot.removeSnapshot(
-        new INode.ReclaimContext(fsdir.getBlockStoragePolicySuite(),
-                                 collectedBlocks, removedINodes, null),
-        snapshotName);
+    srcRoot.removeSnapshot(reclaimContext, snapshotName);
     numSnapshots.getAndDecrement();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaByStorageType.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaByStorageType.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaByStorageType.java
index 6d38937..6703066 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaByStorageType.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaByStorageType.java
@@ -47,7 +47,6 @@ public class TestQuotaByStorageType {
   private static final long seed = 0L;
   private static final Path dir = new Path("/TestQuotaByStorageType");
 
-  private Configuration conf;
   private MiniDFSCluster cluster;
   private FSDirectory fsdir;
   private DistributedFileSystem dfs;
@@ -57,7 +56,7 @@ public class TestQuotaByStorageType {
 
   @Before
   public void setUp() throws Exception {
-    conf = new Configuration();
+    Configuration conf = new Configuration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
 
     // Setup a 3-node cluster and configure
@@ -194,8 +193,8 @@ public class TestQuotaByStorageType {
         .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
     assertEquals(0, storageTypeConsumed);
 
-    QuotaCounts counts = new QuotaCounts.Builder().build();
-    fnode.computeQuotaUsage(fsn.getBlockManager().getStoragePolicySuite(), counts, true);
+    QuotaCounts counts = fnode.computeQuotaUsage(
+        fsn.getBlockManager().getStoragePolicySuite(), true);
     assertEquals(fnode.dumpTreeRecursively().toString(), 0,
         counts.getTypeSpaces().get(StorageType.SSD));
 
@@ -453,8 +452,8 @@ public class TestQuotaByStorageType {
     assertEquals(0, cntAfterDelete.getStorageSpace());
 
     // Validate the computeQuotaUsage()
-    QuotaCounts counts = new QuotaCounts.Builder().build();
-    fnode.computeQuotaUsage(fsn.getBlockManager().getStoragePolicySuite(), counts, true);
+    QuotaCounts counts = fnode.computeQuotaUsage(
+        fsn.getBlockManager().getStoragePolicySuite(), true);
     assertEquals(fnode.dumpTreeRecursively().toString(), 1,
         counts.getNameSpace());
     assertEquals(fnode.dumpTreeRecursively().toString(), 0,
@@ -565,8 +564,8 @@ public class TestQuotaByStorageType {
         .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
     assertEquals(file1Len, ssdConsumed);
 
-    QuotaCounts counts1 = new QuotaCounts.Builder().build();
-    sub1Node.computeQuotaUsage(fsn.getBlockManager().getStoragePolicySuite(), counts1, true);
+    QuotaCounts counts1 = sub1Node.computeQuotaUsage(
+        fsn.getBlockManager().getStoragePolicySuite(), true);
     assertEquals(sub1Node.dumpTreeRecursively().toString(), file1Len,
         counts1.getTypeSpaces().get(StorageType.SSD));
 
@@ -583,8 +582,8 @@ public class TestQuotaByStorageType {
         .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
     assertEquals(0, ssdConsumed);
 
-    QuotaCounts counts2 = new QuotaCounts.Builder().build();
-    sub1Node.computeQuotaUsage(fsn.getBlockManager().getStoragePolicySuite(), counts2, true);
+    QuotaCounts counts2 = sub1Node.computeQuotaUsage(
+        fsn.getBlockManager().getStoragePolicySuite(), true);
     assertEquals(sub1Node.dumpTreeRecursively().toString(), 0,
         counts2.getTypeSpaces().get(StorageType.SSD));
 
@@ -620,7 +619,7 @@ public class TestQuotaByStorageType {
     assertEquals(file1Len, ssdConsumed);
 
     // Truncate file to 1 * BLOCKSIZE
-    int newFile1Len = BLOCKSIZE * 1;
+    int newFile1Len = BLOCKSIZE;
     dfs.truncate(createdFile1, newFile1Len);
 
     // Verify SSD consumed after truncate

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
index 1fc0628..8b9ebea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 import com.google.common.collect.Lists;
-import junit.framework.Assert;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
@@ -26,6 +25,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.QuotaCounts;
+import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
 
@@ -64,7 +64,8 @@ public class TestFileWithSnapshotFeature {
     ArrayList<INode> removedINodes = new ArrayList<>();
     INode.ReclaimContext ctx = new INode.ReclaimContext(
         bsps, collectedBlocks, removedINodes, null);
-    QuotaCounts counts = sf.updateQuotaAndCollectBlocks(ctx, file, diff);
+    sf.updateQuotaAndCollectBlocks(ctx, file, diff);
+    QuotaCounts counts = ctx.quotaDelta().getCountsCopy();
     Assert.assertEquals(0, counts.getStorageSpace());
     Assert.assertTrue(counts.getTypeSpaces().allLessOrEqual(0));
 
@@ -79,7 +80,8 @@ public class TestFileWithSnapshotFeature {
         .thenReturn(Lists.newArrayList(SSD));
     when(bsp.chooseStorageTypes(REPL_3))
         .thenReturn(Lists.newArrayList(DISK));
-    counts = sf.updateQuotaAndCollectBlocks(ctx, file, diff);
+    sf.updateQuotaAndCollectBlocks(ctx, file, diff);
+    counts = ctx.quotaDelta().getCountsCopy();
     Assert.assertEquals((REPL_3 - REPL_1) * BLOCK_SIZE,
                         counts.getStorageSpace());
     Assert.assertEquals(BLOCK_SIZE, counts.getTypeSpaces().get(DISK));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
index 09bd2dc..ab97a3d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
@@ -784,6 +784,7 @@ public class TestRenameWithSnapshots {
     
     // delete foo
     hdfs.delete(foo_dir1, true);
+    restartClusterAndCheckImage(true);
     hdfs.delete(bar2_dir1, true);
     
     // restart the cluster and check fsimage

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
index 97a77ab..cdd655e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
@@ -208,8 +208,7 @@ public class TestSnapshotDeletion {
         q.getNameSpace());
     assertEquals(dirNode.dumpTreeRecursively().toString(), expectedDs,
         q.getStorageSpace());
-    QuotaCounts counts = new QuotaCounts.Builder().build();
-    dirNode.computeQuotaUsage(fsdir.getBlockStoragePolicySuite(), counts, false);
+    QuotaCounts counts = dirNode.computeQuotaUsage(fsdir.getBlockStoragePolicySuite(), false);
     assertEquals(dirNode.dumpTreeRecursively().toString(), expectedNs,
         counts.getNameSpace());
     assertEquals(dirNode.dumpTreeRecursively().toString(), expectedDs,
@@ -248,8 +247,11 @@ public class TestSnapshotDeletion {
     DFSTestUtil.createFile(hdfs, metaChangeFile2, BLOCKSIZE, REPLICATION, seed);
     
     // Case 1: delete deleteDir before taking snapshots
+    hdfs.setQuota(dir, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
+    checkQuotaUsageComputation(dir, 10, BLOCKSIZE * REPLICATION * 4);
     hdfs.delete(deleteDir, true);
-    
+    checkQuotaUsageComputation(dir, 8, BLOCKSIZE * REPLICATION * 3);
+
     // create snapshot s0
     SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
     
@@ -542,7 +544,7 @@ public class TestSnapshotDeletion {
     
     // check 4: no snapshot copy for toDeleteFile
     try {
-      status = hdfs.getFileStatus(toDeleteFile);
+      hdfs.getFileStatus(toDeleteFile);
       fail("should throw FileNotFoundException");
     } catch (FileNotFoundException e) {
       GenericTestUtils.assertExceptionContains("File does not exist: "
@@ -552,7 +554,7 @@ public class TestSnapshotDeletion {
     final Path toDeleteFileInSnapshot = SnapshotTestHelper.getSnapshotPath(dir,
         "s0", toDeleteFile.toString().substring(dir.toString().length()));
     try {
-      status = hdfs.getFileStatus(toDeleteFileInSnapshot);
+      hdfs.getFileStatus(toDeleteFileInSnapshot);
       fail("should throw FileNotFoundException");
     } catch (FileNotFoundException e) {
       GenericTestUtils.assertExceptionContains("File does not exist: "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2c85db8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotManager.java
index b439a28..be14305 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotManager.java
@@ -23,8 +23,6 @@ import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 
-import java.util.ArrayList;
-
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INode;
@@ -75,8 +73,7 @@ public class TestSnapshotManager {
 
     // Delete a snapshot to free up a slot.
     //
-    sm.deleteSnapshot(iip, "", mock(INode.BlocksMapUpdateInfo.class),
-        new ArrayList<INode>());
+    sm.deleteSnapshot(iip, "", mock(INode.ReclaimContext.class));
 
     // Attempt to create a snapshot again. It should still fail due
     // to snapshot ID rollover.


[36/37] hadoop git commit: MAPREDUCE-5708. Duplicate String.format in YarnOutputFiles.getSpillFileForWrite. Contributed by Konstantin Weitz.

Posted by ji...@apache.org.
MAPREDUCE-5708. Duplicate String.format in
YarnOutputFiles.getSpillFileForWrite. Contributed by Konstantin Weitz.


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

Branch: refs/heads/HDFS-7240
Commit: 05ff54c66c49301c4ec2549704d9d459e784572c
Parents: def9136
Author: Devaraj K <de...@apache.org>
Authored: Thu May 14 22:09:54 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Thu May 14 22:09:54 2015 +0530

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/05ff54c6/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 7fe8483..bc1f427 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -422,6 +422,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6366. mapreduce.terasort.final.sync configuration in TeraSort
     doesn't work. (Takuya Fukudome via ozawa)
 
+    MAPREDUCE-5708. Duplicate String.format in YarnOutputFiles.getSpillFileForWrite.
+    (Konstantin Weitz via devaraj)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05ff54c6/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnOutputFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnOutputFiles.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnOutputFiles.java
index e08e093..e099b8f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnOutputFiles.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnOutputFiles.java
@@ -157,8 +157,8 @@ public class YarnOutputFiles extends MapOutputFile {
   public Path getSpillFileForWrite(int spillNumber, long size)
       throws IOException {
     return lDirAlloc.getLocalPathForWrite(
-        String.format(String.format(SPILL_FILE_PATTERN,
-            conf.get(JobContext.TASK_ATTEMPT_ID), spillNumber)), size, conf);
+        String.format(SPILL_FILE_PATTERN,
+            conf.get(JobContext.TASK_ATTEMPT_ID), spillNumber), size, conf);
   }
 
   /**


[23/37] hadoop git commit: YARN-3579. CommonNodeLabelsManager should support NodeLabel instead of string label name when getting node-to-label/label-to-label mappings. (Sunil G via wangda)

Posted by ji...@apache.org.
YARN-3579. CommonNodeLabelsManager should support NodeLabel instead of string label name when getting node-to-label/label-to-label mappings. (Sunil G via wangda)


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

Branch: refs/heads/HDFS-7240
Commit: d4f53fc9631d682cd79ba440aefa6750dcc898be
Parents: 0f95921
Author: Wangda Tan <wa...@apache.org>
Authored: Wed May 13 13:29:09 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed May 13 13:29:09 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../nodelabels/CommonNodeLabelsManager.java     | 149 ++++++++++++++++---
 .../hadoop/yarn/nodelabels/RMNodeLabel.java     |   7 +-
 .../yarn/nodelabels/NodeLabelTestBase.java      |  30 ++++
 .../nodelabels/TestCommonNodeLabelsManager.java |  22 +++
 5 files changed, 186 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4f53fc9/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e5d5ecf..16c2dd9 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -420,6 +420,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3539. Updated timeline server documentation and marked REST APIs evolving.
     (Steve Loughran via zjshen)
 
+    YARN-3579. CommonNodeLabelsManager should support NodeLabel instead of string 
+    label name when getting node-to-label/label-to-label mappings. (Sunil G via wangda)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4f53fc9/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 f2ff0f6..bf34837 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
@@ -64,6 +64,8 @@ public class CommonNodeLabelsManager extends AbstractService {
   private static final int MAX_LABEL_LENGTH = 255;
   public static final Set<String> EMPTY_STRING_SET = Collections
       .unmodifiableSet(new HashSet<String>(0));
+  public static final Set<NodeLabel> EMPTY_NODELABEL_SET = Collections
+      .unmodifiableSet(new HashSet<NodeLabel>(0));
   public static final String ANY = "*";
   public static final Set<String> ACCESS_ANY_LABEL_SET = ImmutableSet.of(ANY);
   private static final Pattern LABEL_PATTERN = Pattern
@@ -716,23 +718,53 @@ public class CommonNodeLabelsManager extends AbstractService {
    * @return nodes to labels map
    */
   public Map<NodeId, Set<String>> getNodeLabels() {
+    Map<NodeId, Set<String>> nodeToLabels =
+        generateNodeLabelsInfoPerNode(String.class);
+    return nodeToLabels;
+  }
+
+  /**
+   * Get mapping of nodes to label info
+   *
+   * @return nodes to labels map
+   */
+  public Map<NodeId, Set<NodeLabel>> getNodeLabelsInfo() {
+    Map<NodeId, Set<NodeLabel>> nodeToLabels =
+        generateNodeLabelsInfoPerNode(NodeLabel.class);
+    return nodeToLabels;
+  }
+
+  @SuppressWarnings("unchecked")
+  private <T> Map<NodeId, Set<T>> generateNodeLabelsInfoPerNode(Class<T> type) {
     try {
       readLock.lock();
-      Map<NodeId, Set<String>> nodeToLabels =
-          new HashMap<NodeId, Set<String>>();
+      Map<NodeId, Set<T>> nodeToLabels = new HashMap<>();
       for (Entry<String, Host> entry : nodeCollections.entrySet()) {
         String hostName = entry.getKey();
         Host host = entry.getValue();
         for (NodeId nodeId : host.nms.keySet()) {
-          Set<String> nodeLabels = getLabelsByNode(nodeId);
-          if (nodeLabels == null || nodeLabels.isEmpty()) {
-            continue;
+          if (type.isAssignableFrom(String.class)) {
+            Set<String> nodeLabels = getLabelsByNode(nodeId);
+            if (nodeLabels == null || nodeLabels.isEmpty()) {
+              continue;
+            }
+            nodeToLabels.put(nodeId, (Set<T>) nodeLabels);
+          } else {
+            Set<NodeLabel> nodeLabels = getLabelsInfoByNode(nodeId);
+            if (nodeLabels == null || nodeLabels.isEmpty()) {
+              continue;
+            }
+            nodeToLabels.put(nodeId, (Set<T>) nodeLabels);
           }
-          nodeToLabels.put(nodeId, nodeLabels);
         }
         if (!host.labels.isEmpty()) {
-          nodeToLabels
-              .put(NodeId.newInstance(hostName, WILDCARD_PORT), host.labels);
+          if (type.isAssignableFrom(String.class)) {
+            nodeToLabels.put(NodeId.newInstance(hostName, WILDCARD_PORT),
+                (Set<T>) host.labels);
+          } else {
+            nodeToLabels.put(NodeId.newInstance(hostName, WILDCARD_PORT),
+                (Set<T>) createNodeLabelFromLabelNames(host.labels));
+          }
         }
       }
       return Collections.unmodifiableMap(nodeToLabels);
@@ -741,6 +773,7 @@ public class CommonNodeLabelsManager extends AbstractService {
     }
   }
 
+
   /**
    * Get mapping of labels to nodes for all the labels.
    *
@@ -765,28 +798,72 @@ public class CommonNodeLabelsManager extends AbstractService {
   public Map<String, Set<NodeId>> getLabelsToNodes(Set<String> labels) {
     try {
       readLock.lock();
-      Map<String, Set<NodeId>> labelsToNodes =
-          new HashMap<String, Set<NodeId>>();
-      for (String label : labels) {
-        if(label.equals(NO_LABEL)) {
-          continue;
-        }
-        RMNodeLabel nodeLabelInfo = labelCollections.get(label);
-        if(nodeLabelInfo != null) {
-          Set<NodeId> nodeIds = nodeLabelInfo.getAssociatedNodeIds();
-          if (!nodeIds.isEmpty()) {
-            labelsToNodes.put(label, nodeIds);
-          }
-        } else {
-          LOG.warn("getLabelsToNodes : Label [" + label + "] cannot be found");
-        }
-      }      
+      Map<String, Set<NodeId>> labelsToNodes = getLabelsToNodesMapping(labels,
+          String.class);
+      return Collections.unmodifiableMap(labelsToNodes);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+
+  /**
+   * Get mapping of labels to nodes for all the labels.
+   *
+   * @return labels to nodes map
+   */
+  public Map<NodeLabel, Set<NodeId>> getLabelsInfoToNodes() {
+    try {
+      readLock.lock();
+      return getLabelsInfoToNodes(labelCollections.keySet());
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Get mapping of labels info to nodes for specified set of labels.
+   *
+   * @param nodelabels
+   *          set of nodelabels for which labels to nodes mapping will be
+   *          returned.
+   * @return labels to nodes map
+   */
+  public Map<NodeLabel, Set<NodeId>> getLabelsInfoToNodes(Set<String> labels) {
+    try {
+      readLock.lock();
+      Map<NodeLabel, Set<NodeId>> labelsToNodes = getLabelsToNodesMapping(
+          labels, NodeLabel.class);
       return Collections.unmodifiableMap(labelsToNodes);
     } finally {
       readLock.unlock();
     }
   }
 
+  private <T> Map<T, Set<NodeId>> getLabelsToNodesMapping(Set<String> labels,
+      Class<T> type) {
+    Map<T, Set<NodeId>> labelsToNodes = new HashMap<T, Set<NodeId>>();
+    for (String label : labels) {
+      if (label.equals(NO_LABEL)) {
+        continue;
+      }
+      RMNodeLabel nodeLabelInfo = labelCollections.get(label);
+      if (nodeLabelInfo != null) {
+        Set<NodeId> nodeIds = nodeLabelInfo.getAssociatedNodeIds();
+        if (!nodeIds.isEmpty()) {
+          if (type.isAssignableFrom(String.class)) {
+            labelsToNodes.put(type.cast(label), nodeIds);
+          } else {
+            labelsToNodes.put(type.cast(nodeLabelInfo.getNodeLabel()), nodeIds);
+          }
+        }
+      } else {
+        LOG.warn("getLabelsToNodes : Label [" + label + "] cannot be found");
+      }
+    }
+    return labelsToNodes;
+  }
+
   /**
    * Get existing valid labels in repository
    * 
@@ -914,6 +991,30 @@ public class CommonNodeLabelsManager extends AbstractService {
     }
   }
   
+  private Set<NodeLabel> getLabelsInfoByNode(NodeId nodeId) {
+    Set<String> labels = getLabelsByNode(nodeId, nodeCollections);
+    if (labels.isEmpty()) {
+      return EMPTY_NODELABEL_SET;
+    }
+    Set<NodeLabel> nodeLabels = createNodeLabelFromLabelNames(labels);
+    return nodeLabels;
+  }
+
+  private Set<NodeLabel> createNodeLabelFromLabelNames(Set<String> labels) {
+    Set<NodeLabel> nodeLabels = new HashSet<NodeLabel>();
+    for (String label : labels) {
+      if (label.equals(NO_LABEL)) {
+        continue;
+      }
+      RMNodeLabel rmLabel = labelCollections.get(label);
+      if (rmLabel == null) {
+        continue;
+      }
+      nodeLabels.add(rmLabel.getNodeLabel());
+    }
+    return nodeLabels;
+  }
+
   protected void createNodeIfNonExisted(NodeId nodeId) throws IOException {
     Host host = nodeCollections.get(nodeId.getHost());
     if (null == host) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4f53fc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java
index 1c4fd17..feeeaf1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java
@@ -25,7 +25,6 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 public class RMNodeLabel implements Comparable<RMNodeLabel> {
@@ -34,6 +33,7 @@ public class RMNodeLabel implements Comparable<RMNodeLabel> {
   private String labelName;
   private Set<NodeId> nodeIds;
   private boolean exclusive;
+  private NodeLabel nodeLabel;
 
   public RMNodeLabel(NodeLabel nodeLabel) {
     this(nodeLabel.getName(), Resource.newInstance(0, 0), 0,
@@ -52,6 +52,7 @@ public class RMNodeLabel implements Comparable<RMNodeLabel> {
     this.numActiveNMs = activeNMs;
     this.nodeIds = new HashSet<NodeId>();
     this.exclusive = exclusive;
+    this.nodeLabel = NodeLabel.newInstance(labelName, exclusive);
   }
 
   public void addNodeId(NodeId node) {
@@ -100,6 +101,10 @@ public class RMNodeLabel implements Comparable<RMNodeLabel> {
     return new RMNodeLabel(labelName, resource, numActiveNMs, exclusive);
   }
   
+  public NodeLabel getNodeLabel() {
+    return this.nodeLabel;
+  }
+
   @Override
   public int compareTo(RMNodeLabel o) {
     // We should always put empty label entry first after sorting

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4f53fc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/NodeLabelTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/NodeLabelTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/NodeLabelTestBase.java
index 4d406fd..8301d96 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/NodeLabelTestBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/NodeLabelTestBase.java
@@ -26,6 +26,7 @@ import java.util.Set;
 import java.util.Map.Entry;
 
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.junit.Assert;
 
 import com.google.common.collect.ImmutableMap;
@@ -41,6 +42,15 @@ public class NodeLabelTestBase {
     }
   }
 
+  public static void assertLabelInfoMapEquals(Map<NodeId, Set<NodeLabel>> m1,
+      ImmutableMap<NodeId, Set<NodeLabel>> m2) {
+    Assert.assertEquals(m1.size(), m2.size());
+    for (NodeId k : m1.keySet()) {
+      Assert.assertTrue(m2.containsKey(k));
+      assertNLCollectionEquals(m1.get(k), m2.get(k));
+    }
+  }
+
   public static void assertLabelsToNodesEquals(Map<String, Set<NodeId>> m1,
       ImmutableMap<String, Set<NodeId>> m2) {
     Assert.assertEquals(m1.size(), m2.size());
@@ -88,6 +98,14 @@ public class NodeLabelTestBase {
     Assert.assertTrue(s1.containsAll(s2));
   }
 
+  public static void assertNLCollectionEquals(Collection<NodeLabel> c1,
+      Collection<NodeLabel> c2) {
+    Set<NodeLabel> s1 = new HashSet<NodeLabel>(c1);
+    Set<NodeLabel> s2 = new HashSet<NodeLabel>(c2);
+    Assert.assertEquals(s1, s2);
+    Assert.assertTrue(s1.containsAll(s2));
+  }
+
   @SuppressWarnings("unchecked")
   public static <E> Set<E> toSet(E... elements) {
     Set<E> set = Sets.newHashSet(elements);
@@ -105,4 +123,16 @@ public class NodeLabelTestBase {
       return NodeId.newInstance(str, CommonNodeLabelsManager.WILDCARD_PORT);
     }
   }
+
+  public static void assertLabelsInfoToNodesEquals(
+      Map<NodeLabel, Set<NodeId>> m1, ImmutableMap<NodeLabel, Set<NodeId>> m2) {
+    Assert.assertEquals(m1.size(), m2.size());
+    for (NodeLabel k : m1.keySet()) {
+      Assert.assertTrue(m2.containsKey(k));
+      Set<NodeId> s1 = new HashSet<NodeId>(m1.get(k));
+      Set<NodeId> s2 = new HashSet<NodeId>(m2.get(k));
+      Assert.assertEquals(s1, s2);
+      Assert.assertTrue(s1.containsAll(s2));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4f53fc9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java
index 09838b4..c25b0bf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java
@@ -579,4 +579,26 @@ public class TestCommonNodeLabelsManager extends NodeLabelTestBase {
         labelsByNode);
     Assert.assertTrue(labelsByNode.contains("p1"));
   }
+
+  @Test(timeout = 5000)
+  public void testLabelsInfoToNodes() throws IOException {
+    mgr.addToCluserNodeLabels(Arrays.asList(NodeLabel.newInstance("p1", false),
+        NodeLabel.newInstance("p2", true), NodeLabel.newInstance("p3", true)));
+    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n1"), toSet("p1")));
+    Map<NodeLabel, Set<NodeId>> labelsToNodes = mgr.getLabelsInfoToNodes();
+    assertLabelsInfoToNodesEquals(labelsToNodes, ImmutableMap.of(
+        NodeLabel.newInstance("p1", false), toSet(toNodeId("n1"))));
+  }
+
+  @Test(timeout = 5000)
+  public void testGetNodeLabelsInfo() throws IOException {
+    mgr.addToCluserNodeLabels(Arrays.asList(NodeLabel.newInstance("p1", false),
+        NodeLabel.newInstance("p2", true), NodeLabel.newInstance("p3", false)));
+    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n1"), toSet("p2")));
+    mgr.addLabelsToNode(ImmutableMap.of(toNodeId("n2"), toSet("p3")));
+
+    assertLabelInfoMapEquals(mgr.getNodeLabelsInfo(), ImmutableMap.of(
+        toNodeId("n1"), toSet(NodeLabel.newInstance("p2", true)),
+        toNodeId("n2"), toSet(NodeLabel.newInstance("p3", false))));
+  }
 }


[06/37] hadoop git commit: HDFS-8255. Rename getBlockReplication to getPreferredBlockReplication. (Contributed by Zhe Zhang)

Posted by ji...@apache.org.
HDFS-8255. Rename getBlockReplication to getPreferredBlockReplication. (Contributed by Zhe Zhang)


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

Branch: refs/heads/HDFS-7240
Commit: 6d5da9484185ca9f585195d6da069b9cd5be4044
Parents: 8badd82
Author: yliu <yl...@apache.org>
Authored: Tue May 12 21:29:22 2015 +0800
Committer: yliu <yl...@apache.org>
Committed: Tue May 12 21:29:22 2015 +0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../server/blockmanagement/BlockCollection.java |  2 +-
 .../blockmanagement/BlockInfoContiguous.java    |  2 +-
 .../server/blockmanagement/BlockManager.java    | 16 ++++++-------
 .../blockmanagement/DecommissionManager.java    | 10 ++++----
 .../hdfs/server/namenode/FSDirAttrOp.java       |  4 ++--
 .../hdfs/server/namenode/FSDirConcatOp.java     |  4 ++--
 .../hdfs/server/namenode/FSDirectory.java       |  4 ++--
 .../hdfs/server/namenode/FSEditLogLoader.java   |  7 +++---
 .../hdfs/server/namenode/FSNamesystem.java      |  4 ++--
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  8 +++----
 .../hdfs/server/namenode/NamenodeFsck.java      |  9 ++++---
 .../snapshot/FileWithSnapshotFeature.java       |  5 ++--
 .../blockmanagement/TestBlockManager.java       |  6 ++---
 .../blockmanagement/TestReplicationPolicy.java  |  4 ++--
 .../snapshot/TestFileWithSnapshotFeature.java   |  2 +-
 .../namenode/snapshot/TestSnapshotDeletion.java |  4 ++--
 .../snapshot/TestSnapshotReplication.java       | 25 ++++++++++++--------
 18 files changed, 66 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/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 7cff8d4..cd477af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -543,6 +543,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8357. Consolidate parameters of INode.CleanSubtree() into a parameter
     objects. (Li Lu via wheat9)
 
+    HDFS-8255. Rename getBlockReplication to getPreferredBlockReplication.
+    (Contributed by Zhe Zhang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
index e9baf85..c0a959c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
@@ -58,7 +58,7 @@ public interface BlockCollection {
    * Get block replication for the collection 
    * @return block replication value
    */
-  public short getBlockReplication();
+  public short getPreferredBlockReplication();
 
   /** 
    * @return the storage policy ID.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
index df27882..1ba3536 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
@@ -380,7 +380,7 @@ public class BlockInfoContiguous extends Block
     if(isComplete()) {
       BlockInfoContiguousUnderConstruction ucBlock =
           new BlockInfoContiguousUnderConstruction(this,
-          getBlockCollection().getBlockReplication(), s, targets);
+          getBlockCollection().getPreferredBlockReplication(), s, targets);
       ucBlock.setBlockCollection(getBlockCollection());
       return ucBlock;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 9d9a631..ab2607b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -1173,7 +1173,7 @@ public class BlockManager {
       return;
     } 
     short expectedReplicas =
-        b.corrupted.getBlockCollection().getBlockReplication();
+        b.corrupted.getBlockCollection().getPreferredBlockReplication();
 
     // Add replica to the data-node if it is not already there
     if (storageInfo != null) {
@@ -1348,7 +1348,7 @@ public class BlockManager {
               continue;
             }
 
-            requiredReplication = bc.getBlockReplication();
+            requiredReplication = bc.getPreferredBlockReplication();
 
             // get a source data-node
             containingNodes = new ArrayList<DatanodeDescriptor>();
@@ -1432,7 +1432,7 @@ public class BlockManager {
             rw.targets = null;
             continue;
           }
-          requiredReplication = bc.getBlockReplication();
+          requiredReplication = bc.getPreferredBlockReplication();
 
           // do not schedule more if enough replicas is already pending
           NumberReplicas numReplicas = countNodes(block);
@@ -2584,7 +2584,7 @@ public class BlockManager {
     }
 
     // handle underReplication/overReplication
-    short fileReplication = bc.getBlockReplication();
+    short fileReplication = bc.getPreferredBlockReplication();
     if (!isNeededReplication(storedBlock, fileReplication, numCurrentReplica)) {
       neededReplications.remove(storedBlock, numCurrentReplica,
           num.decommissionedAndDecommissioning(), fileReplication);
@@ -2815,7 +2815,7 @@ public class BlockManager {
     }
     // calculate current replication
     short expectedReplication =
-        block.getBlockCollection().getBlockReplication();
+        block.getBlockCollection().getPreferredBlockReplication();
     NumberReplicas num = countNodes(block);
     int numCurrentReplica = num.liveReplicas();
     // add to under-replicated queue if need to be
@@ -3316,7 +3316,7 @@ public class BlockManager {
     while(it.hasNext()) {
       final Block block = it.next();
       BlockCollection bc = blocksMap.getBlockCollection(block);
-      short expectedReplication = bc.getBlockReplication();
+      short expectedReplication = bc.getPreferredBlockReplication();
       NumberReplicas num = countNodes(block);
       int numCurrentReplica = num.liveReplicas();
       if (numCurrentReplica > expectedReplication) {
@@ -3430,7 +3430,7 @@ public class BlockManager {
    * process it as an over replicated block.
    */
   public void checkReplication(BlockCollection bc) {
-    final short expected = bc.getBlockReplication();
+    final short expected = bc.getPreferredBlockReplication();
     for (Block block : bc.getBlocks()) {
       final NumberReplicas n = countNodes(block);
       if (isNeededReplication(block, expected, n.liveReplicas())) { 
@@ -3469,7 +3469,7 @@ public class BlockManager {
    */
   private int getReplication(Block block) {
     final BlockCollection bc = blocksMap.getBlockCollection(block);
-    return bc == null? 0: bc.getBlockReplication();
+    return bc == null? 0: bc.getPreferredBlockReplication();
   }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
index 5c9aec7..5f7366e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
@@ -251,7 +251,7 @@ public class DecommissionManager {
   private boolean isSufficientlyReplicated(BlockInfoContiguous block, 
       BlockCollection bc,
       NumberReplicas numberReplicas) {
-    final int numExpected = bc.getBlockReplication();
+    final int numExpected = bc.getPreferredBlockReplication();
     final int numLive = numberReplicas.liveReplicas();
     if (!blockManager.isNeededReplication(block, numExpected, numLive)) {
       // Block doesn't need replication. Skip.
@@ -288,7 +288,7 @@ public class DecommissionManager {
       DatanodeDescriptor srcNode, NumberReplicas num,
       Iterable<DatanodeStorageInfo> storages) {
     int curReplicas = num.liveReplicas();
-    int curExpectedReplicas = bc.getBlockReplication();
+    int curExpectedReplicas = bc.getPreferredBlockReplication();
     StringBuilder nodeList = new StringBuilder();
     for (DatanodeStorageInfo storage : storages) {
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
@@ -564,8 +564,8 @@ public class DecommissionManager {
 
         // Schedule under-replicated blocks for replication if not already
         // pending
-        if (blockManager.isNeededReplication(block, bc.getBlockReplication(),
-            liveReplicas)) {
+        if (blockManager.isNeededReplication(block,
+            bc.getPreferredBlockReplication(), liveReplicas)) {
           if (!blockManager.neededReplications.contains(block) &&
               blockManager.pendingReplications.getNumReplicas(block) == 0 &&
               namesystem.isPopulatingReplQueues()) {
@@ -573,7 +573,7 @@ public class DecommissionManager {
             blockManager.neededReplications.add(block,
                 curReplicas,
                 num.decommissionedAndDecommissioning(),
-                bc.getBlockReplication());
+                bc.getPreferredBlockReplication());
           }
         }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index d01e2c8..879738d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -387,7 +387,7 @@ public class FSDirAttrOp {
       return null;
     }
     INodeFile file = inode.asFile();
-    final short oldBR = file.getBlockReplication();
+    final short oldBR = file.getPreferredBlockReplication();
 
     // before setFileReplication, check for increasing block replication.
     // if replication > oldBR, then newBR == replication.
@@ -399,7 +399,7 @@ public class FSDirAttrOp {
 
     file.setFileReplication(replication, iip.getLatestSnapshotId());
 
-    final short newBR = file.getBlockReplication();
+    final short newBR = file.getPreferredBlockReplication();
     // check newBR < oldBR case.
     if (newBR < oldBR) {
       long dsDelta = file.storagespaceConsumed(null).getStorageSpace() / newBR;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
index 31a6af7..3f22f51 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
@@ -167,9 +167,9 @@ class FSDirConcatOp {
   private static QuotaCounts computeQuotaDeltas(FSDirectory fsd,
       INodeFile target, INodeFile[] srcList) {
     QuotaCounts deltas = new QuotaCounts.Builder().build();
-    final short targetRepl = target.getBlockReplication();
+    final short targetRepl = target.getPreferredBlockReplication();
     for (INodeFile src : srcList) {
-      short srcRepl = src.getBlockReplication();
+      short srcRepl = src.getPreferredBlockReplication();
       long fileSize = src.computeFileSize();
       if (targetRepl != srcRepl) {
         deltas.addStorageSpace(fileSize * (targetRepl - srcRepl));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index b289c39..c981626 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -489,7 +489,7 @@ public class FSDirectory implements Closeable {
 
       // check quota limits and updated space consumed
       updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
-          fileINode.getBlockReplication(), true);
+          fileINode.getPreferredBlockReplication(), true);
 
       // associate new last block for the file
       BlockInfoContiguousUnderConstruction blockInfo =
@@ -546,7 +546,7 @@ public class FSDirectory implements Closeable {
 
     // update space consumed
     updateCount(iip, 0, -fileNode.getPreferredBlockSize(),
-        fileNode.getBlockReplication(), true);
+        fileNode.getPreferredBlockReplication(), true);
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 7964188..b7a4870 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -971,7 +971,7 @@ public class FSEditLogLoader {
     }
     // add the new block
     BlockInfoContiguous newBI = new BlockInfoContiguousUnderConstruction(
-          newBlock, file.getBlockReplication());
+          newBlock, file.getPreferredBlockReplication());
     fsNamesys.getBlockManager().addBlockCollection(newBI, file);
     file.addBlock(newBI);
     fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
@@ -1050,13 +1050,14 @@ public class FSEditLogLoader {
           // what about an old-version fsync() where fsync isn't called
           // until several blocks in?
           newBI = new BlockInfoContiguousUnderConstruction(
-              newBlock, file.getBlockReplication());
+              newBlock, file.getPreferredBlockReplication());
         } else {
           // OP_CLOSE should add finalized blocks. This code path
           // is only executed when loading edits written by prior
           // versions of Hadoop. Current versions always log
           // OP_ADD operations as each block is allocated.
-          newBI = new BlockInfoContiguous(newBlock, file.getBlockReplication());
+          newBI = new BlockInfoContiguous(newBlock,
+              file.getPreferredBlockReplication());
         }
         fsNamesys.getBlockManager().addBlockCollection(newBI, file);
         file.addBlock(newBI);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 9e30812..33aaa72 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -2106,7 +2106,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       // Add new truncateBlock into blocksMap and
       // use oldBlock as a source for copy-on-truncate recovery
       truncatedBlockUC = new BlockInfoContiguousUnderConstruction(newBlock,
-          file.getBlockReplication());
+          file.getPreferredBlockReplication());
       truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
       truncatedBlockUC.setTruncateBlock(oldBlock);
       file.setLastBlock(truncatedBlockUC, blockManager.getStorages(oldBlock));
@@ -2807,7 +2807,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final BlockInfoContiguous lastBlock = file.getLastBlock();
     if (lastBlock != null) {
       final long diff = file.getPreferredBlockSize() - lastBlock.getNumBytes();
-      final short repl = file.getBlockReplication();
+      final short repl = file.getPreferredBlockReplication();
       delta.addStorageSpace(diff * repl);
       final BlockStoragePolicy policy = dir.getBlockStoragePolicySuite()
           .getPolicy(file.getStoragePolicyID());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 3790c74..44f23bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -355,7 +355,7 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   @Override // BlockCollection
-  public short getBlockReplication() {
+  public short getPreferredBlockReplication() {
     short max = getFileReplication(CURRENT_STATE_ID);
     FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
     if (sf != null) {
@@ -728,7 +728,7 @@ public class INodeFile extends INodeWithAdditionalFields
       blocks = allBlocks;
     }
 
-    final short replication = getBlockReplication();
+    final short replication = getPreferredBlockReplication();
     for (BlockInfoContiguous b : blocks) {
       long blockSize = b.isComplete() ? b.getNumBytes() :
           getPreferredBlockSize();
@@ -850,10 +850,10 @@ public class INodeFile extends INodeWithAdditionalFields
         truncatedBytes -= bi.getNumBytes();
       }
 
-      delta.addStorageSpace(-truncatedBytes * getBlockReplication());
+      delta.addStorageSpace(-truncatedBytes * getPreferredBlockReplication());
       if (bsps != null) {
         List<StorageType> types = bsps.chooseStorageTypes(
-            getBlockReplication());
+            getPreferredBlockReplication());
         for (StorageType t : types) {
           if (t.supportTypeQuota()) {
             delta.addTypeSpace(t, -truncatedBytes);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 11e89c9..61f8fdb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -257,15 +257,18 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       NumberReplicas numberReplicas= bm.countNodes(block);
       out.println("Block Id: " + blockId);
       out.println("Block belongs to: "+iNode.getFullPathName());
-      out.println("No. of Expected Replica: " + bc.getBlockReplication());
+      out.println("No. of Expected Replica: " +
+          bc.getPreferredBlockReplication());
       out.println("No. of live Replica: " + numberReplicas.liveReplicas());
       out.println("No. of excess Replica: " + numberReplicas.excessReplicas());
-      out.println("No. of stale Replica: " + numberReplicas.replicasOnStaleNodes());
+      out.println("No. of stale Replica: " +
+          numberReplicas.replicasOnStaleNodes());
       out.println("No. of decommissioned Replica: "
           + numberReplicas.decommissioned());
       out.println("No. of decommissioning Replica: "
           + numberReplicas.decommissioning());
-      out.println("No. of corrupted Replica: " + numberReplicas.corruptReplicas());
+      out.println("No. of corrupted Replica: " +
+          numberReplicas.corruptReplicas());
       //record datanodes that have corrupted block replica
       Collection<DatanodeDescriptor> corruptionRecord = null;
       if (bm.getCorruptReplicas(block) != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
index 3bb549b..213c186 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
@@ -151,11 +151,12 @@ public class FileWithSnapshotFeature implements INode.Feature {
     long oldStoragespace;
     if (removed.snapshotINode != null) {
       short replication = removed.snapshotINode.getFileReplication();
-      short currentRepl = file.getBlockReplication();
+      short currentRepl = file.getPreferredBlockReplication();
       if (replication > currentRepl) {
         long oldFileSizeNoRep = currentRepl == 0
             ? file.computeFileSize(true, true)
-            : oldCounts.getStorageSpace() / file.getBlockReplication();
+            : oldCounts.getStorageSpace() /
+            file.getPreferredBlockReplication();
         oldStoragespace = oldFileSizeNoRep * replication;
         oldCounts.setStorageSpace(oldStoragespace);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index 1e09e19..58210c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -433,7 +433,7 @@ public class TestBlockManager {
   
   private BlockInfoContiguous addBlockOnNodes(long blockId, List<DatanodeDescriptor> nodes) {
     BlockCollection bc = Mockito.mock(BlockCollection.class);
-    Mockito.doReturn((short)3).when(bc).getBlockReplication();
+    Mockito.doReturn((short)3).when(bc).getPreferredBlockReplication();
     BlockInfoContiguous blockInfo = blockOnNodes(blockId, nodes);
 
     bm.blocksMap.addBlockCollection(blockInfo, bc);
@@ -740,7 +740,7 @@ public class TestBlockManager {
     BlockInfoContiguous blockInfo =
         new BlockInfoContiguous(block, (short) 3);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
-    Mockito.doReturn((short) 3).when(bc).getBlockReplication();
+    Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication();
     bm.blocksMap.addBlockCollection(blockInfo, bc);
     return blockInfo;
   }
@@ -750,7 +750,7 @@ public class TestBlockManager {
     BlockInfoContiguousUnderConstruction blockInfo =
         new BlockInfoContiguousUnderConstruction(block, (short) 3);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
-    Mockito.doReturn((short) 3).when(bc).getBlockReplication();
+    Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication();
     bm.blocksMap.addBlockCollection(blockInfo, bc);
     return blockInfo;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index 1d6dad8..f117ef7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@ -1171,7 +1171,7 @@ public class TestReplicationPolicy {
     // queue.
     BlockInfoContiguousUnderConstruction info = new BlockInfoContiguousUnderConstruction(block1, (short) 1);
     BlockCollection bc = mock(BlockCollection.class);
-    when(bc.getBlockReplication()).thenReturn((short)1);
+    when(bc.getPreferredBlockReplication()).thenReturn((short)1);
     bm.addBlockCollection(info, bc);
 
     // Adding this block will increase its current replication, and that will
@@ -1215,7 +1215,7 @@ public class TestReplicationPolicy {
     final BlockCollection mbc = mock(BlockCollection.class);
     when(mbc.getLastBlock()).thenReturn(info);
     when(mbc.getPreferredBlockSize()).thenReturn(block1.getNumBytes() + 1);
-    when(mbc.getBlockReplication()).thenReturn((short)1);
+    when(mbc.getPreferredBlockReplication()).thenReturn((short)1);
     when(mbc.isUnderConstruction()).thenReturn(true);
     ContentSummary cs = mock(ContentSummary.class);
     when(cs.getLength()).thenReturn((long)1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
index 831d65d..1fc0628 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
@@ -70,7 +70,7 @@ public class TestFileWithSnapshotFeature {
 
     // INode only exists in the snapshot
     INodeFile snapshotINode = mock(INodeFile.class);
-    when(file.getBlockReplication()).thenReturn(REPL_1);
+    when(file.getPreferredBlockReplication()).thenReturn(REPL_1);
     Whitebox.setInternalState(snapshotINode, "header", (long) REPL_3 << 48);
     Whitebox.setInternalState(diff, "snapshotINode", snapshotINode);
     when(diff.getSnapshotINode()).thenReturn(snapshotINode);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
index a679183..97a77ab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
@@ -841,12 +841,12 @@ public class TestSnapshotDeletion {
     }
     
     INodeFile nodeFile13 = (INodeFile) fsdir.getINode(file13.toString());
-    assertEquals(REPLICATION_1, nodeFile13.getBlockReplication());
+    assertEquals(REPLICATION_1, nodeFile13.getPreferredBlockReplication());
     TestSnapshotBlocksMap.assertBlockCollection(file13.toString(), 1, fsdir,
         blockmanager);
     
     INodeFile nodeFile12 = (INodeFile) fsdir.getINode(file12_s1.toString());
-    assertEquals(REPLICATION_1, nodeFile12.getBlockReplication());
+    assertEquals(REPLICATION_1, nodeFile12.getPreferredBlockReplication());
   }
   
   /** Test deleting snapshots with modification on the metadata of directory */ 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d5da948/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotReplication.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotReplication.java
index 5264cb7..4eac634 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotReplication.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotReplication.java
@@ -40,7 +40,7 @@ import org.junit.Test;
 /**
  * This class tests the replication handling/calculation of snapshots. In
  * particular, {@link INodeFile#getFileReplication()} and
- * {@link INodeFile#getBlockReplication()} are tested to make sure
+ * {@link INodeFile#getPreferredBlockReplication()} are tested to make sure
  * the number of replication is calculated correctly with/without snapshots.
  */
 public class TestSnapshotReplication {
@@ -81,7 +81,7 @@ public class TestSnapshotReplication {
   /**
    * Check the replication of a given file. We test both
    * {@link INodeFile#getFileReplication()} and
-   * {@link INodeFile#getBlockReplication()}.
+   * {@link INodeFile#getPreferredBlockReplication()}.
    *
    * @param file The given file
    * @param replication The expected replication number
@@ -95,10 +95,11 @@ public class TestSnapshotReplication {
     // INodeFile#getFileReplication().
     short fileReplication = hdfs.getFileStatus(file1).getReplication();
     assertEquals(replication, fileReplication);
-    // Check the correctness of getBlockReplication()
+    // Check the correctness of getPreferredBlockReplication()
     INode inode = fsdir.getINode(file1.toString());
     assertTrue(inode instanceof INodeFile);
-    assertEquals(blockReplication, ((INodeFile) inode).getBlockReplication());
+    assertEquals(blockReplication,
+        ((INodeFile) inode).getPreferredBlockReplication());
   }
   
   /**
@@ -137,16 +138,19 @@ public class TestSnapshotReplication {
    */
   private void checkSnapshotFileReplication(Path currentFile,
       Map<Path, Short> snapshotRepMap, short expectedBlockRep) throws Exception {
-    // First check the getBlockReplication for the INode of the currentFile
+    // First check the getPreferredBlockReplication for the INode of
+    // the currentFile
     final INodeFile inodeOfCurrentFile = getINodeFile(currentFile);
-    assertEquals(expectedBlockRep, inodeOfCurrentFile.getBlockReplication());
+    assertEquals(expectedBlockRep,
+        inodeOfCurrentFile.getPreferredBlockReplication());
     // Then check replication for every snapshot
     for (Path ss : snapshotRepMap.keySet()) {
       final INodesInPath iip = fsdir.getINodesInPath(ss.toString(), true);
       final INodeFile ssInode = iip.getLastINode().asFile();
       // The replication number derived from the
-      // INodeFileWithLink#getBlockReplication should always == expectedBlockRep
-      assertEquals(expectedBlockRep, ssInode.getBlockReplication());
+      // INodeFileWithLink#getPreferredBlockReplication should
+      // always == expectedBlockRep
+      assertEquals(expectedBlockRep, ssInode.getPreferredBlockReplication());
       // Also check the number derived from INodeFile#getFileReplication
       assertEquals(snapshotRepMap.get(ss).shortValue(),
           ssInode.getFileReplication(iip.getPathSnapshotId()));
@@ -218,8 +222,9 @@ public class TestSnapshotReplication {
     for (Path ss : snapshotRepMap.keySet()) {
       final INodeFile ssInode = getINodeFile(ss);
       // The replication number derived from the
-      // INodeFileWithLink#getBlockReplication should always == expectedBlockRep
-      assertEquals(REPLICATION, ssInode.getBlockReplication());
+      // INodeFileWithLink#getPreferredBlockReplication should
+      // always == expectedBlockRep
+      assertEquals(REPLICATION, ssInode.getPreferredBlockReplication());
       // Also check the number derived from INodeFile#getFileReplication
       assertEquals(snapshotRepMap.get(ss).shortValue(),
           ssInode.getFileReplication());


[16/37] hadoop git commit: HADOOP-9723. Improve error message when hadoop archive output path already exists. Contributed by Jean-Baptiste Onofré and Yongjun Zhang.

Posted by ji...@apache.org.
HADOOP-9723. Improve error message when hadoop archive output path already exists. Contributed by Jean-Baptiste Onofré and Yongjun Zhang.


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

Branch: refs/heads/HDFS-7240
Commit: 92c38e41e1fffb9d60d4fa5d4d2212777af9e9a5
Parents: e82067b
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed May 13 17:28:57 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed May 13 17:28:57 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 .../org/apache/hadoop/tools/HadoopArchives.java | 13 ++++--
 .../apache/hadoop/tools/TestHadoopArchives.java | 44 +++++++++++++++++++-
 3 files changed, 54 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/92c38e41/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 a15444e..2fbecbb 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -560,6 +560,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11948. test-patch's issue matching regex should be configurable.
     (Sean Busbey via aw)
 
+    HADOOP-9723. Improve error message when hadoop archive output path already
+    exists. (Jean-Baptiste Onofré and Yongjun Zhang via aajisak)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92c38e41/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java b/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java
index c5c42b1..f00bb6d 100644
--- a/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java
+++ b/hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java
@@ -469,8 +469,13 @@ public class HadoopArchives implements Tool {
     Path outputPath = new Path(dest, archiveName);
     FileOutputFormat.setOutputPath(conf, outputPath);
     FileSystem outFs = outputPath.getFileSystem(conf);
-    if (outFs.exists(outputPath) || outFs.isFile(dest)) {
-      throw new IOException("Invalid Output: " + outputPath);
+    if (outFs.exists(outputPath)) {
+      throw new IOException("Archive path: "
+          + outputPath.toString() + " already exists");
+    }
+    if (outFs.isFile(dest)) {
+      throw new IOException("Destination " + dest.toString()
+          + " should be a directory but is a file");
     }
     conf.set(DST_DIR_LABEL, outputPath.toString());
     Path stagingArea;
@@ -846,8 +851,8 @@ public class HadoopArchives implements Tool {
           Path argPath = new Path(args[i]);
           if (argPath.isAbsolute()) {
             System.out.println(usage);
-            throw new IOException("source path " + argPath +
-                " is not relative  to "+ parentPath);
+            throw new IOException("Source path " + argPath +
+                " is not relative to "+ parentPath);
           }
           srcPaths.add(new Path(parentPath, argPath));
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92c38e41/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java b/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java
index 3fa5919..101cb06 100644
--- a/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java
+++ b/hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.tools;
 import java.io.ByteArrayOutputStream;
 import java.io.FilterInputStream;
 import java.io.IOException;
+import java.io.OutputStream;
 import java.io.PrintStream;
 import java.net.URI;
 import java.util.ArrayList;
@@ -175,8 +176,47 @@ public class TestHadoopArchives {
     final List<String> harPaths = lsr(shell, fullHarPathStr);
     Assert.assertEquals(originalPaths, harPaths);
   }
-  
-@Test
+
+  @Test
+  public void testOutputPathValidity() throws Exception {
+    final String inputPathStr = inputPath.toUri().getPath();
+    final URI uri = fs.getUri();
+    final String harName = "foo.har";
+    System.setProperty(HadoopArchives.TEST_HADOOP_ARCHIVES_JAR_PATH,
+        HADOOP_ARCHIVES_JAR);
+    final HadoopArchives har = new HadoopArchives(conf);
+
+    PrintStream stderr = System.err;
+    ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
+    PrintStream newErr = new PrintStream(byteStream);
+    System.setErr(newErr);
+
+    // fail if the archive path already exists
+    createFile(archivePath, fs, harName);
+    final String[] args = { "-archiveName", harName, "-p", inputPathStr, "*",
+        archivePath.toString() };
+    Assert.assertEquals(-1, ToolRunner.run(har, args));
+    String output = byteStream.toString();
+    final Path outputPath = new Path(archivePath, harName);
+    Assert.assertTrue(output.indexOf("Archive path: " + outputPath.toString()
+        + " already exists") != -1);
+
+    byteStream.reset();
+
+    // fail if the destination directory is a file
+    createFile(archivePath, fs, "sub1");
+    final Path archivePath2 = new Path(archivePath, "sub1");
+    final String[] args2 = { "-archiveName", harName, "-p", inputPathStr, "*",
+        archivePath2.toString() };
+    Assert.assertEquals(-1, ToolRunner.run(har, args2));
+    output = byteStream.toString();
+    Assert.assertTrue(output.indexOf("Destination " + archivePath2.toString()
+        + " should be a directory but is a file") != -1);
+
+    System.setErr(stderr);
+  }
+
+  @Test
   public void testPathWithSpaces() throws Exception {
     // create files/directories with spaces
     createFile(inputPath, fs, "c c");