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 xg...@apache.org on 2017/02/22 00:06:29 UTC

[01/50] [abbrv] hadoop git commit: YARN-5966. AMRMClient changes to support ExecutionType update. (asuresh) [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/YARN-5734 b59c20d17 -> 6e1a54403 (forced update)


YARN-5966. AMRMClient changes to support ExecutionType update. (asuresh)


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

Branch: refs/heads/YARN-5734
Commit: aaf106fde35ec97e2e2ea4d7a67434038c4273ac
Parents: 4164a20
Author: Arun Suresh <as...@apache.org>
Authored: Tue Feb 14 06:08:27 2017 -0800
Committer: Arun Suresh <as...@apache.org>
Committed: Tue Feb 14 06:09:10 2017 -0800

----------------------------------------------------------------------
 .../yarn/api/records/UpdateContainerError.java  |  19 +-
 .../src/main/proto/yarn_service_protos.proto    |   1 +
 .../hadoop/yarn/client/api/AMRMClient.java      |  33 +-
 .../yarn/client/api/async/AMRMClientAsync.java  |  33 +-
 .../api/async/impl/AMRMClientAsyncImpl.java     |   7 +-
 .../yarn/client/api/impl/AMRMClientImpl.java    | 111 +++--
 .../yarn/client/api/impl/TestAMRMClient.java    |  60 ++-
 .../api/impl/TestAMRMClientOnRMRestart.java     |   8 +-
 .../TestOpportunisticContainerAllocation.java   | 400 +++++++++++++++++--
 .../impl/pb/UpdateContainerErrorPBImpl.java     |  16 +
 .../server/resourcemanager/RMServerUtils.java   |  14 +-
 ...pportunisticContainerAllocatorAMService.java |   5 +-
 .../capacity/TestIncreaseAllocationExpirer.java |   4 +-
 13 files changed, 587 insertions(+), 124 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaf106fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/UpdateContainerError.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/UpdateContainerError.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/UpdateContainerError.java
index e7458cf..4d184cb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/UpdateContainerError.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/UpdateContainerError.java
@@ -59,6 +59,22 @@ public abstract class UpdateContainerError {
   public abstract void setReason(String reason);
 
   /**
+   * Get current container version.
+   * @return Current container Version.
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Unstable
+  public abstract int getCurrentContainerVersion();
+
+  /**
+   * Set current container version.
+   * @param currentVersion Current container version.
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Unstable
+  public abstract void setCurrentContainerVersion(int currentVersion);
+
+  /**
    * Get the {@code UpdateContainerRequest} that was not satisfiable.
    * @return UpdateContainerRequest
    */
@@ -89,6 +105,7 @@ public abstract class UpdateContainerError {
   @Override
   public String toString() {
     return "UpdateContainerError{reason=" + getReason() + ", "
+        + "currentVersion=" + getCurrentContainerVersion() + ", "
         + "req=" + getUpdateContainerRequest() + "}";
   }
 
@@ -120,6 +137,6 @@ public abstract class UpdateContainerError {
     } else if (!req.equals(other.getUpdateContainerRequest())) {
       return false;
     }
-    return true;
+    return getCurrentContainerVersion() == other.getCurrentContainerVersion();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaf106fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index df3c852..c6647c8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -78,6 +78,7 @@ message UpdateContainerRequestProto {
 message UpdateContainerErrorProto {
   optional string reason = 1;
   optional UpdateContainerRequestProto update_request = 2;
+  optional int32 current_container_version = 3;
 }
 
 message AllocateRequestProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaf106fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
index 52155f5..15d0065 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
@@ -33,17 +33,20 @@ import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.yarn.util.resource.Resources;
 
 @InterfaceAudience.Public
 @InterfaceStability.Stable
@@ -518,12 +521,38 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
    * ResourceManager to change the existing resource allocation to the target
    * resource allocation.
    *
+   * @deprecated use
+   * {@link #requestContainerUpdate(Container, UpdateContainerRequest)}
+   *
    * @param container The container returned from the last successful resource
    *                  allocation or resource change
    * @param capability  The target resource capability of the container
    */
-  public abstract void requestContainerResourceChange(
-      Container container, Resource capability);
+  @Deprecated
+  public void requestContainerResourceChange(
+      Container container, Resource capability) {
+    Preconditions.checkNotNull(container, "Container cannot be null!!");
+    Preconditions.checkNotNull(capability,
+        "UpdateContainerRequest cannot be null!!");
+    requestContainerUpdate(container, UpdateContainerRequest.newInstance(
+        container.getVersion(), container.getId(),
+        Resources.fitsIn(capability, container.getResource()) ?
+            ContainerUpdateType.DECREASE_RESOURCE :
+            ContainerUpdateType.INCREASE_RESOURCE,
+        capability, null));
+  }
+
+  /**
+   * Request a container update before calling <code>allocate</code>.
+   * Any previous pending update request of the same container will be
+   * removed.
+   *
+   * @param container The container returned from the last successful resource
+   *                  allocation or update
+   * @param updateContainerRequest The <code>UpdateContainerRequest</code>.
+   */
+  public abstract void requestContainerUpdate(
+      Container container, UpdateContainerRequest updateContainerRequest);
 
   /**
    * Release containers assigned by the Resource Manager. If the app cannot use

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaf106fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
index d2195a6..4cb27cd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
@@ -36,11 +36,13 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
@@ -50,6 +52,7 @@ import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.yarn.util.resource.Resources;
 
 /**
  * <code>AMRMClientAsync</code> handles communication with the ResourceManager
@@ -284,12 +287,38 @@ extends AbstractService {
    * ResourceManager to change the existing resource allocation to the target
    * resource allocation.
    *
+   * @deprecated use
+   * {@link #requestContainerUpdate(Container, UpdateContainerRequest)}
+   *
    * @param container The container returned from the last successful resource
    *                  allocation or resource change
    * @param capability  The target resource capability of the container
    */
-  public abstract void requestContainerResourceChange(
-      Container container, Resource capability);
+  @Deprecated
+  public void requestContainerResourceChange(
+      Container container, Resource capability) {
+    Preconditions.checkNotNull(container, "Container cannot be null!!");
+    Preconditions.checkNotNull(capability,
+        "UpdateContainerRequest cannot be null!!");
+    requestContainerUpdate(container, UpdateContainerRequest.newInstance(
+        container.getVersion(), container.getId(),
+        Resources.fitsIn(capability, container.getResource()) ?
+            ContainerUpdateType.DECREASE_RESOURCE :
+            ContainerUpdateType.INCREASE_RESOURCE,
+        capability, null));
+  }
+
+  /**
+   * Request a container update before calling <code>allocate</code>.
+   * Any previous pending update request of the same container will be
+   * removed.
+   *
+   * @param container The container returned from the last successful resource
+   *                  allocation or update
+   * @param updateContainerRequest The <code>UpdateContainerRequest</code>.
+   */
+  public abstract void requestContainerUpdate(
+      Container container, UpdateContainerRequest updateContainerRequest);
 
   /**
    * Release containers assigned by the Resource Manager. If the app cannot use

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaf106fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
index 3e72d3f..9e2c0e5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
@@ -207,9 +208,9 @@ extends AMRMClientAsync<T> {
   }
 
   @Override
-  public void requestContainerResourceChange(
-      Container container, Resource capability) {
-    client.requestContainerResourceChange(container, capability);
+  public void requestContainerUpdate(Container container,
+      UpdateContainerRequest updateContainerRequest) {
+    client.requestContainerUpdate(container, updateContainerRequest);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaf106fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
index 44fc1e0..7da91de 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
@@ -169,15 +169,16 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
   protected Set<ContainerId> pendingRelease = new TreeSet<ContainerId>();
   // change map holds container resource change requests between two allocate()
   // calls, and are cleared after each successful allocate() call.
-  protected final Map<ContainerId, SimpleEntry<Container, Resource>> change =
-      new HashMap<>();
+  protected final Map<ContainerId,
+      SimpleEntry<Container, UpdateContainerRequest>> change = new HashMap<>();
   // pendingChange map holds history of container resource change requests in
   // case AM needs to reregister with the ResourceManager.
   // Change requests are removed from this map if RM confirms the change
   // through allocate response, or if RM confirms that the container has been
   // completed.
-  protected final Map<ContainerId, SimpleEntry<Container, Resource>>
-      pendingChange = new HashMap<>();
+  protected final Map<ContainerId,
+      SimpleEntry<Container, UpdateContainerRequest>> pendingChange =
+      new HashMap<>();
 
   public AMRMClientImpl() {
     super(AMRMClientImpl.class.getName());
@@ -259,7 +260,7 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
     AllocateRequest allocateRequest = null;
     List<String> blacklistToAdd = new ArrayList<String>();
     List<String> blacklistToRemove = new ArrayList<String>();
-    Map<ContainerId, SimpleEntry<Container, Resource>> oldChange =
+    Map<ContainerId, SimpleEntry<Container, UpdateContainerRequest>> oldChange =
         new HashMap<>();
     try {
       synchronized (this) {
@@ -374,14 +375,15 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
           //
           // Only insert entries from the cached oldChange map
           // that do not exist in the current change map:
-          for (Map.Entry<ContainerId, SimpleEntry<Container, Resource>> entry :
+          for (Map.Entry<ContainerId,
+              SimpleEntry<Container, UpdateContainerRequest>> entry :
               oldChange.entrySet()) {
             ContainerId oldContainerId = entry.getKey();
             Container oldContainer = entry.getValue().getKey();
-            Resource oldResource = entry.getValue().getValue();
+            UpdateContainerRequest oldupdate = entry.getValue().getValue();
             if (change.get(oldContainerId) == null) {
               change.put(
-                  oldContainerId, new SimpleEntry<>(oldContainer, oldResource));
+                  oldContainerId, new SimpleEntry<>(oldContainer, oldupdate));
             }
           }
           blacklistAdditions.addAll(blacklistToAdd);
@@ -394,19 +396,17 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
 
   private List<UpdateContainerRequest> createUpdateList() {
     List<UpdateContainerRequest> updateList = new ArrayList<>();
-    for (Map.Entry<ContainerId, SimpleEntry<Container, Resource>> entry :
-        change.entrySet()) {
-      Resource targetCapability = entry.getValue().getValue();
-      Resource currCapability = entry.getValue().getKey().getResource();
-      int version = entry.getValue().getKey().getVersion();
+    for (Map.Entry<ContainerId, SimpleEntry<Container,
+        UpdateContainerRequest>> entry : change.entrySet()) {
+      Resource targetCapability = entry.getValue().getValue().getCapability();
+      ExecutionType targetExecType =
+          entry.getValue().getValue().getExecutionType();
       ContainerUpdateType updateType =
-          ContainerUpdateType.INCREASE_RESOURCE;
-      if (Resources.fitsIn(targetCapability, currCapability)) {
-        updateType = ContainerUpdateType.DECREASE_RESOURCE;
-      }
+          entry.getValue().getValue().getContainerUpdateType();
+      int version = entry.getValue().getKey().getVersion();
       updateList.add(
           UpdateContainerRequest.newInstance(version, entry.getKey(),
-              updateType, targetCapability, null));
+              updateType, targetCapability, targetExecType));
     }
     return updateList;
   }
@@ -591,21 +591,47 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
   }
 
   @Override
-  public synchronized void requestContainerResourceChange(
-      Container container, Resource capability) {
-    validateContainerResourceChangeRequest(
-        container.getId(), container.getResource(), capability);
+  public synchronized void requestContainerUpdate(
+      Container container, UpdateContainerRequest updateContainerRequest) {
+    Preconditions.checkNotNull(container, "Container cannot be null!!");
+    Preconditions.checkNotNull(updateContainerRequest,
+        "UpdateContainerRequest cannot be null!!");
+    LOG.info("Requesting Container update : " +
+        "container=" + container + ", " +
+        "updateType=" + updateContainerRequest.getContainerUpdateType() + ", " +
+        "targetCapability=" + updateContainerRequest.getCapability() + ", " +
+        "targetExecType=" + updateContainerRequest.getExecutionType());
+    if (updateContainerRequest.getCapability() != null &&
+        updateContainerRequest.getExecutionType() == null) {
+      validateContainerResourceChangeRequest(
+          updateContainerRequest.getContainerUpdateType(),
+          container.getId(), container.getResource(),
+          updateContainerRequest.getCapability());
+    } else if (updateContainerRequest.getExecutionType() != null &&
+        updateContainerRequest.getCapability() == null) {
+      validateContainerExecTypeChangeRequest(
+          updateContainerRequest.getContainerUpdateType(),
+          container.getId(), container.getExecutionType(),
+          updateContainerRequest.getExecutionType());
+    } else if (updateContainerRequest.getExecutionType() == null &&
+        updateContainerRequest.getCapability() == null) {
+      throw new IllegalArgumentException("Both target Capability and" +
+          "target Execution Type are null");
+    } else {
+      throw new IllegalArgumentException("Support currently exists only for" +
+          " EITHER update of Capability OR update of Execution Type NOT both");
+    }
     if (change.get(container.getId()) == null) {
       change.put(container.getId(),
-          new SimpleEntry<>(container, capability));
+          new SimpleEntry<>(container, updateContainerRequest));
     } else {
-      change.get(container.getId()).setValue(capability);
+      change.get(container.getId()).setValue(updateContainerRequest);
     }
     if (pendingChange.get(container.getId()) == null) {
       pendingChange.put(container.getId(),
-          new SimpleEntry<>(container, capability));
+          new SimpleEntry<>(container, updateContainerRequest));
     } else {
-      pendingChange.get(container.getId()).setValue(capability);
+      pendingChange.get(container.getId()).setValue(updateContainerRequest);
     }
   }
 
@@ -755,7 +781,8 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
   }
 
   private void validateContainerResourceChangeRequest(
-      ContainerId containerId, Resource original, Resource target) {
+      ContainerUpdateType updateType, ContainerId containerId,
+      Resource original, Resource target) {
     Preconditions.checkArgument(containerId != null,
         "ContainerId cannot be null");
     Preconditions.checkArgument(original != null,
@@ -768,6 +795,36 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
     Preconditions.checkArgument(!Resources.equals(Resources.none(), target)
             && Resources.fitsIn(Resources.none(), target),
         "Target resource capability must be greater than 0");
+    if (ContainerUpdateType.DECREASE_RESOURCE == updateType) {
+      Preconditions.checkArgument(Resources.fitsIn(target, original),
+          "Target resource capability must fit in Original capability");
+    } else {
+      Preconditions.checkArgument(Resources.fitsIn(original, target),
+          "Target resource capability must be more than Original capability");
+
+    }
+  }
+
+  private void validateContainerExecTypeChangeRequest(
+      ContainerUpdateType updateType, ContainerId containerId,
+      ExecutionType original, ExecutionType target) {
+    Preconditions.checkArgument(containerId != null,
+        "ContainerId cannot be null");
+    Preconditions.checkArgument(original != null,
+        "Original Execution Type cannot be null");
+    Preconditions.checkArgument(target != null,
+        "Target Execution Type cannot be null");
+    if (ContainerUpdateType.DEMOTE_EXECUTION_TYPE == updateType) {
+      Preconditions.checkArgument(target == ExecutionType.OPPORTUNISTIC
+              && original == ExecutionType.GUARANTEED,
+          "Incorrect Container update request, target should be" +
+              " OPPORTUNISTIC and original should be GUARANTEED");
+    } else {
+      Preconditions.checkArgument(target == ExecutionType.GUARANTEED
+                  && original == ExecutionType.OPPORTUNISTIC,
+          "Incorrect Container update request, target should be" +
+              " GUARANTEED and original should be OPPORTUNISTIC");
+    }
   }
 
   private void addResourceRequestToAsk(ResourceRequest remoteRequest) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaf106fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
index 8b1bbc7..4f73bac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
@@ -51,29 +51,7 @@ import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
-import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.ContainerState;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ExecutionType;
-import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.NMToken;
-import org.apache.hadoop.yarn.api.records.NodeReport;
-import org.apache.hadoop.yarn.api.records.NodeState;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.api.records.Token;
-import org.apache.hadoop.yarn.api.records.UpdatedContainer;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.api.records.*;
 import org.apache.hadoop.yarn.client.ClientRMProxy;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
@@ -1058,26 +1036,36 @@ public class TestAMRMClient {
         (AMRMClientImpl<ContainerRequest>) amClient;
     Assert.assertEquals(0, amClientImpl.change.size());
     // verify newer request overwrites older request for the container1
-    amClientImpl.requestContainerResourceChange(
-        container1, Resource.newInstance(2048, 1));
-    amClientImpl.requestContainerResourceChange(
-        container1, Resource.newInstance(4096, 1));
+    amClientImpl.requestContainerUpdate(container1,
+        UpdateContainerRequest.newInstance(container1.getVersion(),
+            container1.getId(), ContainerUpdateType.INCREASE_RESOURCE,
+            Resource.newInstance(2048, 1), null));
+    amClientImpl.requestContainerUpdate(container1,
+        UpdateContainerRequest.newInstance(container1.getVersion(),
+            container1.getId(), ContainerUpdateType.INCREASE_RESOURCE,
+            Resource.newInstance(4096, 1), null));
     Assert.assertEquals(Resource.newInstance(4096, 1),
-        amClientImpl.change.get(container1.getId()).getValue());
+        amClientImpl.change.get(container1.getId()).getValue().getCapability());
     // verify new decrease request cancels old increase request for container1
-    amClientImpl.requestContainerResourceChange(
-        container1, Resource.newInstance(512, 1));
+    amClientImpl.requestContainerUpdate(container1,
+        UpdateContainerRequest.newInstance(container1.getVersion(),
+            container1.getId(), ContainerUpdateType.DECREASE_RESOURCE,
+            Resource.newInstance(512, 1), null));
     Assert.assertEquals(Resource.newInstance(512, 1),
-        amClientImpl.change.get(container1.getId()).getValue());
+        amClientImpl.change.get(container1.getId()).getValue().getCapability());
     // request resource increase for container2
-    amClientImpl.requestContainerResourceChange(
-        container2, Resource.newInstance(2048, 1));
+    amClientImpl.requestContainerUpdate(container2,
+        UpdateContainerRequest.newInstance(container2.getVersion(),
+            container2.getId(), ContainerUpdateType.INCREASE_RESOURCE,
+            Resource.newInstance(2048, 1), null));
     Assert.assertEquals(Resource.newInstance(2048, 1),
-        amClientImpl.change.get(container2.getId()).getValue());
+        amClientImpl.change.get(container2.getId()).getValue().getCapability());
     // verify release request will cancel pending change requests for the same
     // container
-    amClientImpl.requestContainerResourceChange(
-        container3, Resource.newInstance(2048, 1));
+    amClientImpl.requestContainerUpdate(container3,
+        UpdateContainerRequest.newInstance(container3.getVersion(),
+            container3.getId(), ContainerUpdateType.INCREASE_RESOURCE,
+            Resource.newInstance(2048, 1), null));
     Assert.assertEquals(3, amClientImpl.pendingChange.size());
     amClientImpl.releaseAssignedContainer(container3.getId());
     Assert.assertEquals(2, amClientImpl.pendingChange.size());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaf106fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
index ac77446..39a7633 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -233,8 +234,11 @@ public class TestAMRMClientOnRMRestart {
     nm1.nodeHeartbeat(containerId.getApplicationAttemptId(),
         containerId.getContainerId(), ContainerState.RUNNING);
     dispatcher.await();
-    amClient.requestContainerResourceChange(
-        container, Resource.newInstance(2048, 1));
+    amClient.requestContainerUpdate(
+        container, UpdateContainerRequest.newInstance(
+            container.getVersion(), container.getId(),
+            ContainerUpdateType.INCREASE_RESOURCE,
+            Resource.newInstance(2048, 1), null));
     it.remove();
 
     allocateResponse = amClient.allocate(0.3f);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaf106fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestOpportunisticContainerAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestOpportunisticContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestOpportunisticContainerAllocation.java
index 802c207..305d18b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestOpportunisticContainerAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestOpportunisticContainerAllocation.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
@@ -44,6 +45,8 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.client.ClientRMProxy;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
@@ -54,6 +57,9 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.After;
@@ -66,13 +72,17 @@ import org.junit.Test;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 
 /**
  * Class that tests the allocation of OPPORTUNISTIC containers through the
@@ -83,7 +93,6 @@ public class TestOpportunisticContainerAllocation {
   private static MiniYARNCluster yarnCluster = null;
   private static YarnClient yarnClient = null;
   private static List<NodeReport> nodeReports = null;
-  private static ApplicationAttemptId attemptId = null;
   private static int nodeCount = 3;
 
   private static final int ROLLING_INTERVAL_SEC = 13;
@@ -92,12 +101,22 @@ public class TestOpportunisticContainerAllocation {
   private static Resource capability;
   private static Priority priority;
   private static Priority priority2;
+  private static Priority priority3;
+  private static Priority priority4;
   private static String node;
   private static String rack;
   private static String[] nodes;
   private static String[] racks;
   private final static int DEFAULT_ITERATION = 3;
 
+  // Per test..
+  private ApplicationAttemptId attemptId = null;
+  private AMRMClientImpl<AMRMClient.ContainerRequest> amClient = null;
+  private long availMB;
+  private int availVCores;
+  private long allocMB;
+  private int allocVCores;
+
   @BeforeClass
   public static void setup() throws Exception {
     // start minicluster
@@ -106,7 +125,7 @@ public class TestOpportunisticContainerAllocation {
         YarnConfiguration.RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS,
         ROLLING_INTERVAL_SEC);
     conf.setLong(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, AM_EXPIRE_MS);
-    conf.setInt(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS, 100);
+    conf.setInt(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS, 1000);
     // set the minimum allocation so that resource decrease can go under 1024
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 512);
     conf.setBoolean(
@@ -129,7 +148,9 @@ public class TestOpportunisticContainerAllocation {
 
     priority = Priority.newInstance(1);
     priority2 = Priority.newInstance(2);
-    capability = Resource.newInstance(1024, 1);
+    priority3 = Priority.newInstance(3);
+    priority4 = Priority.newInstance(4);
+    capability = Resource.newInstance(512, 1);
 
     node = nodeReports.get(0).getNodeId().getHost();
     rack = nodeReports.get(0).getRackName();
@@ -193,10 +214,35 @@ public class TestOpportunisticContainerAllocation {
     UserGroupInformation.getCurrentUser().addToken(appAttempt.getAMRMToken());
     appAttempt.getAMRMToken()
         .setService(ClientRMProxy.getAMRMTokenService(conf));
+
+    // start am rm client
+    amClient = (AMRMClientImpl<AMRMClient.ContainerRequest>)AMRMClient
+        .createAMRMClient();
+
+    //setting an instance NMTokenCache
+    amClient.setNMTokenCache(new NMTokenCache());
+    //asserting we are not using the singleton instance cache
+    Assert.assertNotSame(NMTokenCache.getSingleton(),
+        amClient.getNMTokenCache());
+
+    amClient.init(conf);
+    amClient.start();
+
+    amClient.registerApplicationMaster("Host", 10000, "");
   }
 
   @After
   public void cancelApp() throws YarnException, IOException {
+    try {
+      amClient
+          .unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, null,
+              null);
+    } finally {
+      if (amClient != null &&
+          amClient.getServiceState() == Service.STATE.STARTED) {
+        amClient.stop();
+      }
+    }
     yarnClient.killApplication(attemptId.getApplicationId());
     attemptId = null;
   }
@@ -214,43 +260,254 @@ public class TestOpportunisticContainerAllocation {
   }
 
   @Test(timeout = 60000)
-  public void testAMRMClient() throws YarnException, IOException {
-    AMRMClient<AMRMClient.ContainerRequest> amClient = null;
+  public void testPromotionFromAcquired() throws YarnException, IOException {
+    // setup container request
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    amClient.addContainerRequest(
+        new AMRMClient.ContainerRequest(capability, null, null, priority2, 0,
+            true, null,
+            ExecutionTypeRequest.newInstance(
+                ExecutionType.OPPORTUNISTIC, true)));
+
+    int oppContainersRequestedAny =
+        amClient.getTable(0).get(priority2, ResourceRequest.ANY,
+            ExecutionType.OPPORTUNISTIC, capability).remoteRequest
+            .getNumContainers();
+
+    assertEquals(1, oppContainersRequestedAny);
+
+    assertEquals(1, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // RM should allocate container within 2 calls to allocate()
+    int allocatedContainerCount = 0;
+    Map<ContainerId, Container> allocatedOpportContainers = new HashMap<>();
+    int iterationsLeft = 50;
+
+    amClient.getNMTokenCache().clearCache();
+    Assert.assertEquals(0,
+        amClient.getNMTokenCache().numberOfTokensInCache());
+    HashMap<String, Token> receivedNMTokens = new HashMap<>();
+
+    updateMetrics("Before Opp Allocation");
+
+    while (allocatedContainerCount < oppContainersRequestedAny
+        && iterationsLeft-- > 0) {
+      AllocateResponse allocResponse = amClient.allocate(0.1f);
+      assertEquals(0, amClient.ask.size());
+      assertEquals(0, amClient.release.size());
+
+      allocatedContainerCount +=
+          allocResponse.getAllocatedContainers().size();
+      for (Container container : allocResponse.getAllocatedContainers()) {
+        if (container.getExecutionType() == ExecutionType.OPPORTUNISTIC) {
+          allocatedOpportContainers.put(container.getId(), container);
+          removeCR(container);
+        }
+      }
+
+      for (NMToken token : allocResponse.getNMTokens()) {
+        String nodeID = token.getNodeId().toString();
+        receivedNMTokens.put(nodeID, token.getToken());
+      }
+
+      if (allocatedContainerCount < oppContainersRequestedAny) {
+        // sleep to let NM's heartbeat to RM and trigger allocations
+        sleep(100);
+      }
+    }
+
+    assertEquals(oppContainersRequestedAny, allocatedContainerCount);
+    assertEquals(oppContainersRequestedAny, allocatedOpportContainers.size());
+
+    updateMetrics("After Opp Allocation / Before Promotion");
+
     try {
-      // start am rm client
-      amClient = AMRMClient.<AMRMClient.ContainerRequest>createAMRMClient();
+      Container c = allocatedOpportContainers.values().iterator().next();
+      amClient.requestContainerUpdate(
+          c, UpdateContainerRequest.newInstance(c.getVersion(),
+              c.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
+              null, ExecutionType.OPPORTUNISTIC));
+      Assert.fail("Should throw Exception..");
+    } catch (IllegalArgumentException e) {
+      System.out.println("## " + e.getMessage());
+      Assert.assertTrue(e.getMessage().contains(
+          "target should be GUARANTEED and original should be OPPORTUNISTIC"));
+    }
 
-      //setting an instance NMTokenCache
-      amClient.setNMTokenCache(new NMTokenCache());
-      //asserting we are not using the singleton instance cache
-      Assert.assertNotSame(NMTokenCache.getSingleton(),
-          amClient.getNMTokenCache());
+    Container c = allocatedOpportContainers.values().iterator().next();
+    amClient.requestContainerUpdate(
+        c, UpdateContainerRequest.newInstance(c.getVersion(),
+            c.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
+            null, ExecutionType.GUARANTEED));
+    iterationsLeft = 120;
+    Map<ContainerId, UpdatedContainer> updatedContainers = new HashMap<>();
+    // do a few iterations to ensure RM is not going to send new containers
+    while (iterationsLeft-- > 0 && updatedContainers.isEmpty()) {
+      // inform RM of rejection
+      AllocateResponse allocResponse = amClient.allocate(0.1f);
+      // RM did not send new containers because AM does not need any
+      if (allocResponse.getUpdatedContainers() != null) {
+        for (UpdatedContainer updatedContainer : allocResponse
+            .getUpdatedContainers()) {
+          System.out.println("Got update..");
+          updatedContainers.put(updatedContainer.getContainer().getId(),
+              updatedContainer);
+        }
+      }
+      if (iterationsLeft > 0) {
+        // sleep to make sure NM's heartbeat
+        sleep(100);
+      }
+    }
 
-      amClient.init(conf);
-      amClient.start();
+    updateMetrics("After Promotion");
+
+    assertEquals(1, updatedContainers.size());
+    for (ContainerId cId : allocatedOpportContainers.keySet()) {
+      Container orig = allocatedOpportContainers.get(cId);
+      UpdatedContainer updatedContainer = updatedContainers.get(cId);
+      assertNotNull(updatedContainer);
+      assertEquals(ExecutionType.GUARANTEED,
+          updatedContainer.getContainer().getExecutionType());
+      assertEquals(orig.getResource(),
+          updatedContainer.getContainer().getResource());
+      assertEquals(orig.getNodeId(),
+          updatedContainer.getContainer().getNodeId());
+      assertEquals(orig.getVersion() + 1,
+          updatedContainer.getContainer().getVersion());
+    }
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+    amClient.ask.clear();
+  }
 
-      amClient.registerApplicationMaster("Host", 10000, "");
+  @Test(timeout = 60000)
+  public void testDemotionFromAcquired() throws YarnException, IOException {
+    // setup container request
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
 
-      testOpportunisticAllocation(
-          (AMRMClientImpl<AMRMClient.ContainerRequest>) amClient);
+    amClient.addContainerRequest(
+        new AMRMClient.ContainerRequest(capability, null, null, priority3));
 
-      testAllocation((AMRMClientImpl<AMRMClient.ContainerRequest>)amClient);
+    int guarContainersRequestedAny = amClient.getTable(0).get(priority3,
+        ResourceRequest.ANY, ExecutionType.GUARANTEED, capability)
+        .remoteRequest.getNumContainers();
 
-      amClient
-          .unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, null,
-              null);
+    assertEquals(1, guarContainersRequestedAny);
 
-    } finally {
-      if (amClient != null &&
-          amClient.getServiceState() == Service.STATE.STARTED) {
-        amClient.stop();
+    assertEquals(1, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // RM should allocate container within 2 calls to allocate()
+    int allocatedContainerCount = 0;
+    Map<ContainerId, Container> allocatedGuarContainers = new HashMap<>();
+    int iterationsLeft = 50;
+
+    amClient.getNMTokenCache().clearCache();
+    Assert.assertEquals(0,
+        amClient.getNMTokenCache().numberOfTokensInCache());
+    HashMap<String, Token> receivedNMTokens = new HashMap<>();
+
+    updateMetrics("Before Guar Allocation");
+
+    while (allocatedContainerCount < guarContainersRequestedAny
+        && iterationsLeft-- > 0) {
+      AllocateResponse allocResponse = amClient.allocate(0.1f);
+      assertEquals(0, amClient.ask.size());
+      assertEquals(0, amClient.release.size());
+
+      allocatedContainerCount +=
+          allocResponse.getAllocatedContainers().size();
+      for (Container container : allocResponse.getAllocatedContainers()) {
+        if (container.getExecutionType() == ExecutionType.GUARANTEED) {
+          allocatedGuarContainers.put(container.getId(), container);
+          removeCR(container);
+        }
+      }
+
+      for (NMToken token : allocResponse.getNMTokens()) {
+        String nodeID = token.getNodeId().toString();
+        receivedNMTokens.put(nodeID, token.getToken());
+      }
+
+      if (allocatedContainerCount < guarContainersRequestedAny) {
+        // sleep to let NM's heartbeat to RM and trigger allocations
+        sleep(100);
+      }
+    }
+
+    assertEquals(guarContainersRequestedAny, allocatedContainerCount);
+    assertEquals(guarContainersRequestedAny, allocatedGuarContainers.size());
+
+    updateMetrics("After Guar Allocation / Before Demotion");
+
+    try {
+      Container c = allocatedGuarContainers.values().iterator().next();
+      amClient.requestContainerUpdate(
+          c, UpdateContainerRequest.newInstance(c.getVersion(),
+              c.getId(), ContainerUpdateType.DEMOTE_EXECUTION_TYPE,
+              null, ExecutionType.GUARANTEED));
+      Assert.fail("Should throw Exception..");
+    } catch (IllegalArgumentException e) {
+      System.out.println("## " + e.getMessage());
+      Assert.assertTrue(e.getMessage().contains(
+          "target should be OPPORTUNISTIC and original should be GUARANTEED"));
+    }
+
+    Container c = allocatedGuarContainers.values().iterator().next();
+    amClient.requestContainerUpdate(
+        c, UpdateContainerRequest.newInstance(c.getVersion(),
+            c.getId(), ContainerUpdateType.DEMOTE_EXECUTION_TYPE,
+            null, ExecutionType.OPPORTUNISTIC));
+    iterationsLeft = 120;
+    Map<ContainerId, UpdatedContainer> updatedContainers = new HashMap<>();
+    // do a few iterations to ensure RM is not going to send new containers
+    while (iterationsLeft-- > 0 && updatedContainers.isEmpty()) {
+      // inform RM of rejection
+      AllocateResponse allocResponse = amClient.allocate(0.1f);
+      // RM did not send new containers because AM does not need any
+      if (allocResponse.getUpdatedContainers() != null) {
+        for (UpdatedContainer updatedContainer : allocResponse
+            .getUpdatedContainers()) {
+          System.out.println("Got update..");
+          updatedContainers.put(updatedContainer.getContainer().getId(),
+              updatedContainer);
+        }
+      }
+      if (iterationsLeft > 0) {
+        // sleep to make sure NM's heartbeat
+        sleep(100);
       }
     }
+
+    updateMetrics("After Demotion");
+
+    assertEquals(1, updatedContainers.size());
+    for (ContainerId cId : allocatedGuarContainers.keySet()) {
+      Container orig = allocatedGuarContainers.get(cId);
+      UpdatedContainer updatedContainer = updatedContainers.get(cId);
+      assertNotNull(updatedContainer);
+      assertEquals(ExecutionType.OPPORTUNISTIC,
+          updatedContainer.getContainer().getExecutionType());
+      assertEquals(orig.getResource(),
+          updatedContainer.getContainer().getResource());
+      assertEquals(orig.getNodeId(),
+          updatedContainer.getContainer().getNodeId());
+      assertEquals(orig.getVersion() + 1,
+          updatedContainer.getContainer().getVersion());
+    }
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+    amClient.ask.clear();
   }
 
-  private void testAllocation(
-      final AMRMClientImpl<AMRMClient.ContainerRequest> amClient)
-      throws YarnException, IOException {
+  @Test(timeout = 60000)
+  public void testMixedAllocationAndRelease() throws YarnException,
+      IOException {
     // setup container request
     assertEquals(0, amClient.ask.size());
     assertEquals(0, amClient.release.size());
@@ -274,6 +531,28 @@ public class TestOpportunisticContainerAllocation {
             ExecutionTypeRequest.newInstance(
                 ExecutionType.OPPORTUNISTIC, true)));
 
+    int containersRequestedNode = amClient.getTable(0).get(priority,
+        node, ExecutionType.GUARANTEED, capability).remoteRequest
+        .getNumContainers();
+    int containersRequestedRack = amClient.getTable(0).get(priority,
+        rack, ExecutionType.GUARANTEED, capability).remoteRequest
+        .getNumContainers();
+    int containersRequestedAny = amClient.getTable(0).get(priority,
+        ResourceRequest.ANY, ExecutionType.GUARANTEED, capability)
+        .remoteRequest.getNumContainers();
+    int oppContainersRequestedAny =
+        amClient.getTable(0).get(priority2, ResourceRequest.ANY,
+            ExecutionType.OPPORTUNISTIC, capability).remoteRequest
+            .getNumContainers();
+
+    assertEquals(4, containersRequestedNode);
+    assertEquals(4, containersRequestedRack);
+    assertEquals(4, containersRequestedAny);
+    assertEquals(2, oppContainersRequestedAny);
+
+    assertEquals(4, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
     amClient.removeContainerRequest(
         new AMRMClient.ContainerRequest(capability, nodes, racks, priority));
     amClient.removeContainerRequest(
@@ -284,16 +563,16 @@ public class TestOpportunisticContainerAllocation {
             ExecutionTypeRequest.newInstance(
                 ExecutionType.OPPORTUNISTIC, true)));
 
-    int containersRequestedNode = amClient.getTable(0).get(priority,
+    containersRequestedNode = amClient.getTable(0).get(priority,
         node, ExecutionType.GUARANTEED, capability).remoteRequest
         .getNumContainers();
-    int containersRequestedRack = amClient.getTable(0).get(priority,
+    containersRequestedRack = amClient.getTable(0).get(priority,
         rack, ExecutionType.GUARANTEED, capability).remoteRequest
         .getNumContainers();
-    int containersRequestedAny = amClient.getTable(0).get(priority,
+    containersRequestedAny = amClient.getTable(0).get(priority,
         ResourceRequest.ANY, ExecutionType.GUARANTEED, capability)
         .remoteRequest.getNumContainers();
-    int oppContainersRequestedAny =
+    oppContainersRequestedAny =
         amClient.getTable(0).get(priority2, ResourceRequest.ANY,
             ExecutionType.OPPORTUNISTIC, capability).remoteRequest
             .getNumContainers();
@@ -309,7 +588,7 @@ public class TestOpportunisticContainerAllocation {
     // RM should allocate container within 2 calls to allocate()
     int allocatedContainerCount = 0;
     int allocatedOpportContainerCount = 0;
-    int iterationsLeft = 10;
+    int iterationsLeft = 50;
     Set<ContainerId> releases = new TreeSet<>();
 
     amClient.getNMTokenCache().clearCache();
@@ -324,8 +603,8 @@ public class TestOpportunisticContainerAllocation {
       assertEquals(0, amClient.ask.size());
       assertEquals(0, amClient.release.size());
 
-      allocatedContainerCount += allocResponse.getAllocatedContainers()
-          .size();
+      allocatedContainerCount +=
+          allocResponse.getAllocatedContainers().size();
       for (Container container : allocResponse.getAllocatedContainers()) {
         if (container.getExecutionType() == ExecutionType.OPPORTUNISTIC) {
           allocatedOpportContainerCount++;
@@ -345,9 +624,9 @@ public class TestOpportunisticContainerAllocation {
       }
     }
 
-    assertEquals(allocatedContainerCount,
-        containersRequestedAny + oppContainersRequestedAny);
-    assertEquals(allocatedOpportContainerCount, oppContainersRequestedAny);
+    assertEquals(containersRequestedAny + oppContainersRequestedAny,
+        allocatedContainerCount);
+    assertEquals(oppContainersRequestedAny, allocatedOpportContainerCount);
     for (ContainerId rejectContainerId : releases) {
       amClient.releaseAssignedContainer(rejectContainerId);
     }
@@ -395,26 +674,25 @@ public class TestOpportunisticContainerAllocation {
   /**
    * Tests allocation with requests comprising only opportunistic containers.
    */
-  private void testOpportunisticAllocation(
-      final AMRMClientImpl<AMRMClient.ContainerRequest> amClient)
-      throws YarnException, IOException {
+  @Test(timeout = 60000)
+  public void testOpportunisticAllocation() throws YarnException, IOException {
     // setup container request
     assertEquals(0, amClient.ask.size());
     assertEquals(0, amClient.release.size());
 
     amClient.addContainerRequest(
-        new AMRMClient.ContainerRequest(capability, null, null, priority, 0,
+        new AMRMClient.ContainerRequest(capability, null, null, priority3, 0,
             true, null,
             ExecutionTypeRequest.newInstance(
                 ExecutionType.OPPORTUNISTIC, true)));
     amClient.addContainerRequest(
-        new AMRMClient.ContainerRequest(capability, null, null, priority, 0,
+        new AMRMClient.ContainerRequest(capability, null, null, priority3, 0,
             true, null,
             ExecutionTypeRequest.newInstance(
                 ExecutionType.OPPORTUNISTIC, true)));
 
     int oppContainersRequestedAny =
-        amClient.getTable(0).get(priority, ResourceRequest.ANY,
+        amClient.getTable(0).get(priority3, ResourceRequest.ANY,
             ExecutionType.OPPORTUNISTIC, capability).remoteRequest
             .getNumContainers();
 
@@ -456,9 +734,43 @@ public class TestOpportunisticContainerAllocation {
       }
     }
 
+    assertEquals(oppContainersRequestedAny, allocatedContainerCount);
     assertEquals(1, receivedNMTokens.values().size());
   }
 
+  private void removeCR(Container container) {
+    List<? extends Collection<AMRMClient.ContainerRequest>>
+        matchingRequests = amClient.getMatchingRequests(container
+            .getPriority(),
+        ResourceRequest.ANY, ExecutionType.OPPORTUNISTIC,
+        container.getResource());
+    Set<AMRMClient.ContainerRequest> toRemove = new HashSet<>();
+    for (Collection<AMRMClient.ContainerRequest> rc : matchingRequests) {
+      for (AMRMClient.ContainerRequest cr : rc) {
+        toRemove.add(cr);
+      }
+    }
+    for (AMRMClient.ContainerRequest cr : toRemove) {
+      amClient.removeContainerRequest(cr);
+    }
+  }
+
+  private void updateMetrics(String msg) {
+    AbstractYarnScheduler scheduler =
+        (AbstractYarnScheduler)yarnCluster.getResourceManager()
+            .getResourceScheduler();
+    availMB = scheduler.getRootQueueMetrics().getAvailableMB();
+    availVCores = scheduler.getRootQueueMetrics().getAvailableVirtualCores();
+    allocMB = scheduler.getRootQueueMetrics().getAllocatedMB();
+    allocVCores = scheduler.getRootQueueMetrics().getAllocatedVirtualCores();
+    System.out.println("## METRICS (" + msg + ")==>");
+    System.out.println(" : availMB=" + availMB + ", " +
+        "availVCores=" +availVCores + ", " +
+        "allocMB=" + allocMB + ", " +
+        "allocVCores=" + allocVCores + ", ");
+    System.out.println("<== ##");
+  }
+
   private void sleep(int sleepTime) {
     try {
       Thread.sleep(sleepTime);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaf106fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/UpdateContainerErrorPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/UpdateContainerErrorPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/UpdateContainerErrorPBImpl.java
index fb6c1a7..8ff9d9e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/UpdateContainerErrorPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/UpdateContainerErrorPBImpl.java
@@ -74,6 +74,22 @@ public class UpdateContainerErrorPBImpl extends UpdateContainerError {
   }
 
   @Override
+  public int getCurrentContainerVersion() {
+    YarnServiceProtos.UpdateContainerErrorProtoOrBuilder p =
+        viaProto ? proto : builder;
+    if (!p.hasCurrentContainerVersion()) {
+      return 0;
+    }
+    return p.getCurrentContainerVersion();
+  }
+
+  @Override
+  public void setCurrentContainerVersion(int containerVersion) {
+    maybeInitBuilder();
+    builder.setCurrentContainerVersion(containerVersion);
+  }
+
+  @Override
   public UpdateContainerRequest getUpdateContainerRequest() {
     YarnServiceProtos.UpdateContainerErrorProtoOrBuilder p = viaProto ? proto
         : builder;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaf106fd/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 94bfd58..224a1da 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
@@ -188,19 +188,25 @@ public class RMServerUtils {
           }
         }
       }
-      checkAndcreateUpdateError(updateErrors, updateReq, msg);
+      checkAndcreateUpdateError(updateErrors, updateReq, rmContainer, msg);
     }
     return updateRequests;
   }
 
   private static void checkAndcreateUpdateError(
       List<UpdateContainerError> errors, UpdateContainerRequest updateReq,
-      String msg) {
+      RMContainer rmContainer, String msg) {
     if (msg != null) {
       UpdateContainerError updateError = RECORD_FACTORY
           .newRecordInstance(UpdateContainerError.class);
       updateError.setReason(msg);
       updateError.setUpdateContainerRequest(updateReq);
+      if (rmContainer != null) {
+        updateError.setCurrentContainerVersion(
+            rmContainer.getContainer().getVersion());
+      } else {
+        updateError.setCurrentContainerVersion(-1);
+      }
       errors.add(updateError);
     }
   }
@@ -216,9 +222,7 @@ public class RMServerUtils {
     // version
     if (msg == null && updateReq.getContainerVersion() !=
         rmContainer.getContainer().getVersion()) {
-      msg = INCORRECT_CONTAINER_VERSION_ERROR + "|"
-          + updateReq.getContainerVersion() + "|"
-          + rmContainer.getContainer().getVersion();
+      msg = INCORRECT_CONTAINER_VERSION_ERROR;
     }
     // No more than 1 container update per request.
     if (msg == null &&

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaf106fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.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/TestOpportunisticContainerAllocatorAMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java
index 641ef64..b083642 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java
@@ -251,8 +251,11 @@ public class TestOpportunisticContainerAllocatorAMService {
 
     Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
     Assert.assertEquals(1, allocateResponse.getUpdateErrors().size());
-    Assert.assertEquals("INCORRECT_CONTAINER_VERSION_ERROR|1|0",
+    Assert.assertEquals("INCORRECT_CONTAINER_VERSION_ERROR",
         allocateResponse.getUpdateErrors().get(0).getReason());
+    Assert.assertEquals(0,
+        allocateResponse.getUpdateErrors().get(0)
+            .getCurrentContainerVersion());
     Assert.assertEquals(container.getId(),
         allocateResponse.getUpdateErrors().get(0)
             .getUpdateContainerRequest().getContainerId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaf106fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestIncreaseAllocationExpirer.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/TestIncreaseAllocationExpirer.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/TestIncreaseAllocationExpirer.java
index c5829cf..74cecf2 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/TestIncreaseAllocationExpirer.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/TestIncreaseAllocationExpirer.java
@@ -275,8 +275,10 @@ public class TestIncreaseAllocationExpirer {
             Resources.createResource(5 * GB), null)));
     List<UpdateContainerError> updateErrors = response.getUpdateErrors();
     Assert.assertEquals(1, updateErrors.size());
-    Assert.assertEquals("INCORRECT_CONTAINER_VERSION_ERROR|0|1",
+    Assert.assertEquals("INCORRECT_CONTAINER_VERSION_ERROR",
         updateErrors.get(0).getReason());
+    Assert.assertEquals(1,
+        updateErrors.get(0).getCurrentContainerVersion());
 
     // am1 asks to change containerId2 from 3GB to 5GB
     am1.sendContainerResizingRequest(Collections.singletonList(


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


[15/50] [abbrv] hadoop git commit: HDFS-11333. Print a user friendly error message when plugins are not found. Contributed by Wei-Chiu Chuang.

Posted by xg...@apache.org.
HDFS-11333. Print a user friendly error message when plugins are not found. Contributed by Wei-Chiu Chuang.


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

Branch: refs/heads/YARN-5734
Commit: 859bd159ae554174200334b5eb1d7e8dbef958ad
Parents: ff75132
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Wed Feb 15 02:50:35 2017 -0800
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Wed Feb 15 02:50:35 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/server/datanode/DataNode.java | 10 +++++++++-
 .../org/apache/hadoop/hdfs/server/namenode/NameNode.java | 11 +++++++++--
 2 files changed, 18 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/859bd159/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 9ed80ef..5db41bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -921,7 +921,15 @@ public class DataNode extends ReconfigurableBase
   }
 
   private void startPlugins(Configuration conf) {
-    plugins = conf.getInstances(DFS_DATANODE_PLUGINS_KEY, ServicePlugin.class);
+    try {
+      plugins = conf.getInstances(DFS_DATANODE_PLUGINS_KEY,
+          ServicePlugin.class);
+    } catch (RuntimeException e) {
+      String pluginsValue = conf.get(DFS_DATANODE_PLUGINS_KEY);
+      LOG.error("Unable to load DataNode plugins. Specified list of plugins: " +
+          pluginsValue, e);
+      throw e;
+    }
     for (ServicePlugin p: plugins) {
       try {
         p.start(this);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/859bd159/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index 1752cf7..e7841f0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -809,8 +809,15 @@ public class NameNode extends ReconfigurableBase implements
       httpServer.setFSImage(getFSImage());
     }
     rpcServer.start();
-    plugins = conf.getInstances(DFS_NAMENODE_PLUGINS_KEY,
-        ServicePlugin.class);
+    try {
+      plugins = conf.getInstances(DFS_NAMENODE_PLUGINS_KEY,
+          ServicePlugin.class);
+    } catch (RuntimeException e) {
+      String pluginsValue = conf.get(DFS_NAMENODE_PLUGINS_KEY);
+      LOG.error("Unable to load NameNode plugins. Specified list of plugins: " +
+          pluginsValue, e);
+      throw e;
+    }
     for (ServicePlugin p: plugins) {
       try {
         p.start(this);


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


[14/50] [abbrv] hadoop git commit: YARN-6183. Few missing informations in Application and Application Attempt pages for new YARN UI. Contributed by Akhil PB.

Posted by xg...@apache.org.
YARN-6183. Few missing informations in Application and Application Attempt pages for new YARN UI. Contributed by Akhil PB.


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

Branch: refs/heads/YARN-5734
Commit: ff7513235579bd9ff48d59127864ceb8eda3c509
Parents: b7613e0
Author: Sunil G <su...@apache.org>
Authored: Wed Feb 15 15:52:57 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Wed Feb 15 15:52:57 2017 +0530

----------------------------------------------------------------------
 .../webapp/app/components/app-attempt-table.js  |  9 ++++++++-
 .../src/main/webapp/app/controllers/yarn-app.js |  6 +++---
 .../src/main/webapp/app/models/yarn-app.js      |  2 +-
 .../src/main/webapp/app/serializers/yarn-app.js |  6 +++---
 .../src/main/webapp/app/styles/app.css          | 20 ++++++++++++++------
 .../webapp/app/templates/cluster-overview.hbs   | 10 +++++-----
 .../templates/components/app-attempt-table.hbs  |  6 +++---
 .../templates/components/container-table.hbs    |  6 +++---
 .../src/main/webapp/app/templates/yarn-app.hbs  |  8 ++++----
 9 files changed, 44 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff751323/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/app-attempt-table.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/app-attempt-table.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/app-attempt-table.js
index 4b741b8..3c43037 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/app-attempt-table.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/app-attempt-table.js
@@ -19,4 +19,11 @@
 import Ember from 'ember';
 
 export default Ember.Component.extend({
-});
\ No newline at end of file
+  nodeHttpAddressFormatted: Ember.computed('attempt.nodeHttpAddress', function() {
+    var nodeHttpAddress = this.get('attempt.nodeHttpAddress');
+    if (nodeHttpAddress && nodeHttpAddress.indexOf('://') < 0) {
+      nodeHttpAddress = 'http://' + nodeHttpAddress;
+    }
+    return nodeHttpAddress;
+  })
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff751323/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app.js
index 9c1cb5d..f699a22 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app.js
@@ -35,11 +35,11 @@ export default Ember.Controller.extend({
     }];
   }),
 
-  amHostHttpAddressFormatted: function() {
+  amHostHttpAddressFormatted: Ember.computed('model.app.amHostHttpAddress', function() {
     var amHostAddress = this.get('model.app.amHostHttpAddress');
-    if (amHostAddress.indexOf('http://') < 0) {
+    if (amHostAddress && amHostAddress.indexOf('://') < 0) {
       amHostAddress = 'http://' + amHostAddress;
     }
     return amHostAddress;
-  }.property('model.app.amHostHttpAddress')
+  })
 });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff751323/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app.js
index 1366357..638e5b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app.js
@@ -37,7 +37,7 @@ export default DS.Model.extend({
   amNodeLabelExpression: DS.attr('string'),
   applicationTags: DS.attr('string'),
   applicationType: DS.attr('string'),
-  priority: DS.attr('number'),
+  priority: DS.attr('string'),
   allocatedMB: DS.attr('number'),
   allocatedVCores: DS.attr('number'),
   runningContainers: DS.attr('number'),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff751323/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app.js
index 7c82ec2..83d1e32 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-app.js
@@ -48,13 +48,13 @@ export default DS.JSONAPISerializer.extend({
           finalStatus: payload.finalStatus,
           progress: payload.progress,
           applicationType: payload.applicationType,
-          diagnostics: payload.diagnostics,
+          diagnostics: (payload.diagnostics && payload.diagnostics !== 'null')? payload.diagnostics : '',
           amContainerLogs: payload.amContainerLogs,
           amHostHttpAddress: payload.amHostHttpAddress,
           logAggregationStatus: payload.logAggregationStatus,
-          unmanagedApplication: payload.unmanagedApplication,
+          unmanagedApplication: payload.unmanagedApplication || 'N/A',
           amNodeLabelExpression: payload.amNodeLabelExpression,
-          priority: payload.priority,
+          priority: payload.priority || 'N/A',
           allocatedMB: payload.allocatedMB,
           allocatedVCores: payload.allocatedVCores,
           runningContainers: payload.runningContainers,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff751323/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css
index da5b4bf..eb5fe61 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.css
@@ -126,15 +126,15 @@ text.bar-chart-text {
   fill: Gray;;
 }
 
-div.tooltip {   
-  position: absolute;           
+div.tooltip {
+  position: absolute;
   text-align: center;
   padding: 2px;
   font: 24px sans-serif;
-  background: lightsteelblue;   
-  border: 0px;      
+  background: lightsteelblue;
+  border: 0px;
   border-radius: 8px;
-  pointer-events: none;         
+  pointer-events: none;
 }
 
 /*
@@ -189,7 +189,7 @@ table.dataTable thead .sorting_desc_disabled {
 
 .link {
   fill: none;
-  stroke: #ccc;  
+  stroke: #ccc;
   stroke-width: 2px;
 }
 
@@ -277,3 +277,11 @@ li a.navigation-link.ember-view {
 .x-scroll {
   overflow-x: scroll;
 }
+
+.donut-chart {
+  padding: 0px !important;
+}
+
+.donut-chart svg {
+  width: 100%;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff751323/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/cluster-overview.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/cluster-overview.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/cluster-overview.hbs
index 3bf0f37..ac8f20a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/cluster-overview.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/cluster-overview.hbs
@@ -27,11 +27,11 @@
           <div class="panel-heading">
             Cluster Resource Usage By Applications
           </div>
-          <div class="container-fluid" id="appusage-donut-chart">
+          <div class="container-fluid donut-chart" id="appusage-donut-chart">
             {{app-usage-donut-chart data=model.apps
             showLabels=true
             parentId="appusage-donut-chart"
-            ratio=0.6
+            ratio=0.5
             maxHeight=400}}
           </div>
         </div>
@@ -42,11 +42,11 @@
           <div class="panel-heading">
             Cluster Resource Usage By Leaf Queues
           </div>
-          <div class="container-fluid" id="queueusage-donut-chart">
+          <div class="container-fluid donut-chart" id="queueusage-donut-chart">
             {{queue-usage-donut-chart data=model.queues
             showLabels=true
             parentId="queueusage-donut-chart"
-            ratio=0.6
+            ratio=0.5
             maxHeight=400}}
           </div>
         </div>
@@ -147,4 +147,4 @@
 {{/if}}
 
 
-{{outlet}}
\ No newline at end of file
+{{outlet}}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff751323/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/app-attempt-table.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/app-attempt-table.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/app-attempt-table.hbs
index 2b16f86..fcd076b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/app-attempt-table.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/app-attempt-table.hbs
@@ -33,7 +33,7 @@
     {{#if attempt.IsAmNodeUrl}}
     <tr>
       <td>AM Node Web UI</td>
-      <td><a href={{attempt.nodeHttpAddress}}>{{attempt.nodeHttpAddress}}</a></td>
+      <td><a href="{{nodeHttpAddressFormatted}}" target="_blank">{{nodeHttpAddressFormatted}}</a></td>
     </tr>
     {{/if}}
     <tr>
@@ -43,7 +43,7 @@
     {{#if attempt.IsLinkAvailable}}
     <tr>
       <td>Log</td>
-      <td><a href={{attempt.logsLink}}>link</a></td>
+      <td><a href="{{attempt.logsLink}}" target="_blank">Link</a></td>
     </tr>
     {{/if}}
     {{#if attempt.attemptState}}
@@ -59,4 +59,4 @@
     </tr>
     {{/if}}
   </tbody>
-</table>
\ No newline at end of file
+</table>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff751323/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/container-table.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/container-table.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/container-table.hbs
index 0736a69..586f128 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/container-table.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/container-table.hbs
@@ -36,7 +36,7 @@
     </tr>
     <tr>
       <td>Log</td>
-      <td><a href={{container.logUrl}}>link</a></td>
+      <td><a href="{{container.logUrl}}" target="_blank">Link</a></td>
     </tr>
     <tr>
       <td>Exit Status</td>
@@ -48,7 +48,7 @@
     </tr>
     <tr>
       <td>NodeManager UI</td>
-      <td><a href={{container.nodeHttpAddress}}>{{container.nodeHttpAddress}}</a></td>
+      <td><a href="{{container.nodeHttpAddress}}" target="_blank">{{container.nodeHttpAddress}}</a></td>
     </tr>
   </tbody>
-</table>
\ No newline at end of file
+</table>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff751323/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app.hbs
index 578cc95..8ce8e8b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app.hbs
@@ -165,21 +165,21 @@
 
         <div class="col-md-4 container-fluid">
           <div class="panel panel-default">
-            <div class="panel-heading">App Master Info</div>
+            <div class="panel-heading">Application Master Info</div>
             <table class="display table table-striped table-bordered"
                    cellspacing="0" width="100%">
               <thead>
               <tr>
                 <th>Master Container Log</th>
                 <td>Master Node</td>
-                <td>Master Node Label Expr</td>
+                <td>Master Node Label Expression</td>
               </tr>
               </thead>
 
               <tbody>
               <tr>
-                <td><a href={{model.app.amContainerLogs}} target="_blank">Link</a></td>
-                <td><a href={{amHostHttpAddressFormatted}} target="_blank">Link</a></td>
+                <td><a href="{{model.app.amContainerLogs}}" target="_blank">Link</a></td>
+                <td><a href="{{amHostHttpAddressFormatted}}" target="_blank">Link</a></td>
                 <td>{{model.app.amNodeLabelExpression}}</td>
               </tr>
               </tbody>


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


[38/50] [abbrv] hadoop git commit: YARN-6193. FairScheduler might not trigger preemption when using DRF. (kasha)

Posted by xg...@apache.org.
YARN-6193. FairScheduler might not trigger preemption when using DRF. (kasha)


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

Branch: refs/heads/YARN-5734
Commit: dbbfcf74ab44d7bfdc805b63affd0defc57182b8
Parents: c7a36e6
Author: Karthik Kambatla <ka...@cloudera.com>
Authored: Fri Feb 17 14:07:31 2017 -0800
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Fri Feb 17 14:07:31 2017 -0800

----------------------------------------------------------------------
 .../scheduler/fair/FSAppAttempt.java            |  9 +++--
 .../fair/TestFairSchedulerPreemption.java       | 36 ++++++++++++++------
 2 files changed, 30 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbbfcf74/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
index b1bb9a0..6ed0660 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
@@ -602,12 +602,11 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
 
     // Check if the app's allocation will be over its fairshare even
     // after preempting this container
-    Resource currentUsage = getResourceUsage();
-    Resource fairshare = getFairShare();
-    Resource overFairShareBy = Resources.subtract(currentUsage, fairshare);
+    Resource usageAfterPreemption = Resources.subtract(
+        getResourceUsage(), container.getAllocatedResource());
 
-    return (Resources.fitsIn(container.getAllocatedResource(),
-        overFairShareBy));
+    return !Resources.lessThan(fsQueue.getPolicy().getResourceCalculator(),
+        scheduler.getClusterResource(), usageAfterPreemption, getFairShare());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dbbfcf74/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
index a4d69bf..480a329 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
@@ -57,6 +57,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
   private static final int NODE_CAPACITY_MULTIPLE = 4;
 
   private final boolean fairsharePreemption;
+  private final boolean drf;
 
   // App that takes up the entire cluster
   private FSAppAttempt greedyApp;
@@ -67,13 +68,17 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
   @Parameterized.Parameters(name = "{0}")
   public static Collection<Object[]> getParameters() {
     return Arrays.asList(new Object[][] {
-        {"FairSharePreemption", true},
-        {"MinSharePreemption", false}});
+        {"MinSharePreemption", 0},
+        {"MinSharePreemptionWithDRF", 1},
+        {"FairSharePreemption", 2},
+        {"FairSharePreemptionWithDRF", 3}
+        });
   }
 
-  public TestFairSchedulerPreemption(String name, boolean fairshare)
+  public TestFairSchedulerPreemption(String name, int mode)
       throws IOException {
-    fairsharePreemption = fairshare;
+    fairsharePreemption = (mode > 1); // 2 and 3
+    drf = (mode % 2 == 1); // 1 and 3
     writeAllocFile();
   }
 
@@ -146,6 +151,10 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
 
     out.println("</queue>"); // end of nonpreemptable queue
 
+    if (drf) {
+      out.println("<defaultQueueSchedulingPolicy>drf" +
+          "</defaultQueueSchedulingPolicy>");
+    }
     out.println("</allocations>");
     out.close();
 
@@ -177,9 +186,14 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     scheduler.setClock(clock);
     resourceManager.start();
 
-    // Create and add two nodes to the cluster
-    addNode(NODE_CAPACITY_MULTIPLE * GB, NODE_CAPACITY_MULTIPLE);
-    addNode(NODE_CAPACITY_MULTIPLE * GB, NODE_CAPACITY_MULTIPLE);
+    // Create and add two nodes to the cluster, with capacities
+    // disproportional to the container requests.
+    addNode(NODE_CAPACITY_MULTIPLE * GB, 3 * NODE_CAPACITY_MULTIPLE);
+    addNode(NODE_CAPACITY_MULTIPLE * GB, 3 * NODE_CAPACITY_MULTIPLE);
+
+    // Reinitialize the scheduler so DRF policy picks up cluster capacity
+    // TODO (YARN-6194): One shouldn't need to call this
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     // Verify if child-1 and child-2 are preemptable
     FSQueue child1 =
@@ -257,7 +271,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
 
   private void verifyPreemption() throws InterruptedException {
     // Sleep long enough for four containers to be preempted.
-    for (int i = 0; i < 100; i++) {
+    for (int i = 0; i < 1000; i++) {
       if (greedyApp.getLiveContainers().size() == 4) {
         break;
       }
@@ -265,12 +279,14 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
     }
 
     // Verify the right amount of containers are preempted from greedyApp
-    assertEquals(4, greedyApp.getLiveContainers().size());
+    assertEquals("Incorrect number of containers on the greedy app",
+        4, greedyApp.getLiveContainers().size());
 
     sendEnoughNodeUpdatesToAssignFully();
 
     // Verify the preempted containers are assigned to starvingApp
-    assertEquals(2, starvingApp.getLiveContainers().size());
+    assertEquals("Starved app is not assigned the right number of containers",
+        2, starvingApp.getLiveContainers().size());
   }
 
   private void verifyNoPreemption() throws InterruptedException {


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


[17/50] [abbrv] hadoop git commit: Addendum patch for YARN-6174 - remove deleted file.

Posted by xg...@apache.org.
Addendum patch for YARN-6174 - remove deleted file.


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

Branch: refs/heads/YARN-5734
Commit: 0fc6f38379f0047afd23ac14abcb5086d65a7f67
Parents: ce2d5bf
Author: Junping Du <ju...@apache.org>
Authored: Wed Feb 15 09:07:49 2017 -0800
Committer: Junping Du <ju...@apache.org>
Committed: Wed Feb 15 09:07:49 2017 -0800

----------------------------------------------------------------------
 .../yarn/logaggregation/ContainerLogType.java   | 31 --------------------
 1 file changed, 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0fc6f383/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogType.java
deleted file mode 100644
index c101499..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogType.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.logaggregation;
-
-/**
- * Enumeration of various type of a container log.
- */
-public enum ContainerLogType {
-
-  /** The log is from NodeManager local log directory. */
-  LOCAL,
-
-  /** The log is from Remote FileSystem application log directory. */
-  AGGREGATED
-}


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


[41/50] [abbrv] hadoop git commit: HADOOP-14081. S3A: Consider avoiding array copy in S3ABlockOutputStream (ByteArrayBlock). Contributed by Rajesh Balamohan

Posted by xg...@apache.org.
HADOOP-14081. S3A: Consider avoiding array copy in S3ABlockOutputStream (ByteArrayBlock). Contributed by Rajesh Balamohan


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

Branch: refs/heads/YARN-5734
Commit: 8035749c26947dc641ef87dac041050d439a16d1
Parents: 172b23a
Author: Steve Loughran <st...@apache.org>
Authored: Mon Feb 20 16:21:00 2017 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Feb 20 16:21:46 2017 +0000

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/s3a/S3ADataBlocks.java | 26 +++++++++++++++++---
 1 file changed, 22 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8035749c/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java
index 0fe2af7..05f8efe 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java
@@ -298,6 +298,25 @@ final class S3ADataBlocks {
 
   }
 
+  static class S3AByteArrayOutputStream extends ByteArrayOutputStream {
+
+    S3AByteArrayOutputStream(int size) {
+      super(size);
+    }
+
+    /**
+     * InputStream backed by the internal byte array
+     *
+     * @return
+     */
+    ByteArrayInputStream getInputStream() {
+      ByteArrayInputStream bin = new ByteArrayInputStream(this.buf, 0, count);
+      this.reset();
+      this.buf = null;
+      return bin;
+    }
+  }
+
   /**
    * Stream to memory via a {@code ByteArrayOutputStream}.
    *
@@ -310,14 +329,14 @@ final class S3ADataBlocks {
    */
 
   static class ByteArrayBlock extends DataBlock {
-    private ByteArrayOutputStream buffer;
+    private S3AByteArrayOutputStream buffer;
     private final int limit;
     // cache data size so that it is consistent after the buffer is reset.
     private Integer dataSize;
 
     ByteArrayBlock(int limit) {
       this.limit = limit;
-      buffer = new ByteArrayOutputStream();
+      buffer = new S3AByteArrayOutputStream(limit);
     }
 
     /**
@@ -333,8 +352,7 @@ final class S3ADataBlocks {
     InputStream startUpload() throws IOException {
       super.startUpload();
       dataSize = buffer.size();
-      ByteArrayInputStream bufferData = new ByteArrayInputStream(
-          buffer.toByteArray());
+      ByteArrayInputStream bufferData = buffer.getInputStream();
       buffer = null;
       return bufferData;
     }


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


[29/50] [abbrv] hadoop git commit: YARN-6177. Yarn client should exit with an informative error message if an incompatible Jersey library is used at client. Contributed by Weiwei Yang.

Posted by xg...@apache.org.
YARN-6177. Yarn client should exit with an informative error message if an incompatible Jersey library is used at client. Contributed by Weiwei Yang.


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

Branch: refs/heads/YARN-5734
Commit: 5d339c46f5b16b951afd82afd9e907b9aa2ded9a
Parents: 4fa1afd
Author: Li Lu <gt...@apache.org>
Authored: Thu Feb 16 13:40:26 2017 -0800
Committer: Li Lu <gt...@apache.org>
Committed: Thu Feb 16 13:41:42 2017 -0800

----------------------------------------------------------------------
 .../yarn/client/api/impl/YarnClientImpl.java    |  10 ++
 .../yarn/client/api/impl/TestYarnClient.java    | 165 ++++++++++++++++---
 2 files changed, 155 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d339c46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
index 4a27fee..23b128c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
@@ -381,6 +381,16 @@ public class YarnClientImpl extends YarnClient {
         return null;
       }
       throw e;
+    } catch (NoClassDefFoundError e) {
+      NoClassDefFoundError wrappedError = new NoClassDefFoundError(
+          e.getMessage() + ". It appears that the timeline client "
+              + "failed to initiate because an incompatible dependency "
+              + "in classpath. If timeline service is optional to this "
+              + "client, try to work around by setting "
+              + YarnConfiguration.TIMELINE_SERVICE_ENABLED
+              + " to false in client configuration.");
+      wrappedError.setStackTrace(e.getStackTrace());
+      throw wrappedError;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d339c46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
index 240f31c..c2c9665 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
@@ -156,26 +156,6 @@ public class TestYarnClient {
   }
 
   @Test
-  public void testStartWithTimelineV15Failure() throws Exception{
-    Configuration conf = new Configuration();
-    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
-    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 1.5f);
-    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_CLIENT_BEST_EFFORT,
-        true);
-    YarnClient client = YarnClient.createYarnClient();
-    if(client instanceof YarnClientImpl) {
-      YarnClientImpl impl = (YarnClientImpl) client;
-      YarnClientImpl spyClient = spy(impl);
-      when(spyClient.createTimelineClient()).thenThrow(
-          new IOException("ATS v1.5 client initialization failed. "));
-      spyClient.init(conf);
-      spyClient.start();
-      spyClient.getTimelineDelegationToken();
-      spyClient.stop();
-    }
-  }
-
-  @Test
   public void testStartWithTimelineV15() throws Exception {
     Configuration conf = new Configuration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
@@ -186,6 +166,89 @@ public class TestYarnClient {
     client.stop();
   }
 
+  @Test
+  public void testStartTimelineClientWithErrors()
+      throws Exception {
+    // If timeline client failed to init with a NoClassDefFoundError
+    // it should be wrapped with an informative error message
+    testCreateTimelineClientWithError(
+        1.5f,
+        true,
+        false,
+        new NoClassDefFoundError("Mock a NoClassDefFoundError"),
+        new CreateTimelineClientErrorVerifier(1) {
+          @Override
+          public void verifyError(Throwable e) {
+            Assert.assertTrue(e instanceof NoClassDefFoundError);
+            Assert.assertTrue(e.getMessage() != null &&
+                e.getMessage().contains(
+                    YarnConfiguration.TIMELINE_SERVICE_ENABLED));
+          }
+        });
+
+    // Disable timeline service for this client,
+    // yarn client will not fail when such error happens
+    testCreateTimelineClientWithError(
+        1.5f,
+        false,
+        false,
+        new NoClassDefFoundError("Mock a NoClassDefFoundError"),
+        new CreateTimelineClientErrorVerifier(0) {
+          @Override public void verifyError(Throwable e) {
+            Assert.fail("NoClassDefFoundError while creating timeline client"
+                + "should be tolerated when timeline service is disabled.");
+          }
+        }
+    );
+
+    // Set best-effort to true, verify an error is still fatal
+    testCreateTimelineClientWithError(
+        1.5f,
+        true,
+        true,
+        new NoClassDefFoundError("Mock a NoClassDefFoundError"),
+        new CreateTimelineClientErrorVerifier(1) {
+          @Override public void verifyError(Throwable e) {
+            Assert.assertTrue(e instanceof NoClassDefFoundError);
+            Assert.assertTrue(e.getMessage() != null &&
+                e.getMessage().contains(
+                    YarnConfiguration.TIMELINE_SERVICE_ENABLED));
+          }
+        }
+    );
+
+    // Set best-effort to false, verify that an exception
+    // causes the client to fail
+    testCreateTimelineClientWithError(
+        1.5f,
+        true,
+        false,
+        new IOException("ATS v1.5 client initialization failed."),
+        new CreateTimelineClientErrorVerifier(1) {
+          @Override
+          public void verifyError(Throwable e) {
+            Assert.assertTrue(e instanceof IOException);
+          }
+        }
+    );
+
+    // Set best-effort to true, verify that an normal exception
+    // won't fail the entire client
+    testCreateTimelineClientWithError(
+        1.5f,
+        true,
+        true,
+        new IOException("ATS v1.5 client initialization failed."),
+        new CreateTimelineClientErrorVerifier(0) {
+          @Override
+          public void verifyError(Throwable e) {
+            Assert.fail("IOException while creating timeline client"
+                + "should be tolerated when best effort is true");
+          }
+        }
+    );
+  }
+
   @SuppressWarnings("deprecation")
   @Test (timeout = 30000)
   public void testSubmitApplication() throws Exception {
@@ -1680,4 +1743,66 @@ public class TestYarnClient {
     Assert.assertEquals(containerId, request.getContainerId());
     Assert.assertEquals(command, request.getCommand());
   }
+
+  private void testCreateTimelineClientWithError(
+      float timelineVersion,
+      boolean timelineServiceEnabled,
+      boolean timelineClientBestEffort,
+      Throwable mockErr,
+      CreateTimelineClientErrorVerifier errVerifier) throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
+        timelineServiceEnabled);
+    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_CLIENT_BEST_EFFORT,
+        timelineClientBestEffort);
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION,
+        timelineVersion);
+    YarnClient client = new MockYarnClient();
+    if (client instanceof YarnClientImpl) {
+      YarnClientImpl impl = (YarnClientImpl) client;
+      YarnClientImpl spyClient = spy(impl);
+      when(spyClient.createTimelineClient()).thenThrow(mockErr);
+      CreateTimelineClientErrorVerifier verifier = spy(errVerifier);
+      spyClient.init(conf);
+      spyClient.start();
+
+      ApplicationSubmissionContext context =
+          mock(ApplicationSubmissionContext.class);
+      ContainerLaunchContext containerContext =
+          mock(ContainerLaunchContext.class);
+      ApplicationId applicationId =
+          ApplicationId.newInstance(System.currentTimeMillis(), 1);
+      when(containerContext.getTokens()).thenReturn(null);
+      when(context.getApplicationId()).thenReturn(applicationId);
+      when(spyClient.isSecurityEnabled()).thenReturn(true);
+      when(context.getAMContainerSpec()).thenReturn(containerContext);
+
+      try {
+        spyClient.submitApplication(context);
+      } catch (Throwable e) {
+        verifier.verifyError(e);
+      } finally {
+        // Make sure the verifier runs with expected times
+        // This is required because in case throwable is swallowed
+        // and verifyError never gets the chance to run
+        verify(verifier, times(verifier.getExpectedTimes()))
+            .verifyError(any(Throwable.class));
+        spyClient.stop();
+      }
+    }
+  }
+
+  private abstract class CreateTimelineClientErrorVerifier {
+    // Verify verifyError gets executed with expected times
+    private int times = 0;
+    protected CreateTimelineClientErrorVerifier(int times) {
+      this.times = times;
+    }
+    public int getExpectedTimes() {
+      return this.times;
+    }
+    // Verification a throwable is in desired state
+    // E.g verify type and error message
+    public abstract void verifyError(Throwable e);
+  }
 }


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


[20/50] [abbrv] hadoop git commit: HDFS-11265. Extend visualization for Maintenance Mode under Datanode tab in the NameNode UI. (Marton Elek via mingma)

Posted by xg...@apache.org.
HDFS-11265. Extend visualization for Maintenance Mode under Datanode tab in the NameNode UI. (Marton Elek via mingma)


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

Branch: refs/heads/YARN-5734
Commit: a136936d018b5cebb7aad9a01ea0dcc366e1c3b8
Parents: 0741dd3
Author: Ming Ma <mi...@apache.org>
Authored: Wed Feb 15 20:24:07 2017 -0800
Committer: Ming Ma <mi...@apache.org>
Committed: Wed Feb 15 20:24:07 2017 -0800

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/webapps/static/hadoop.css          | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a136936d/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
index 0901125..341e1f8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
@@ -236,8 +236,8 @@ header.bs-docs-nav, header.bs-docs-nav .navbar-brand {
 }
 
 .dfshealth-node-decommissioned:before {
-    color: #eea236;
-    content: "\e136";
+    color: #bc5f04;
+    content: "\e090";
 }
 
 .dfshealth-node-down:before {
@@ -250,6 +250,11 @@ header.bs-docs-nav, header.bs-docs-nav .navbar-brand {
     content: "\e017";
 }
 
+.dfshealth-node-down-maintenance:before {
+    color: #eea236;
+    content: "\e136";
+}
+
 .dfshealth-node-legend {
     list-style-type: none;
     text-align: right;


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


[31/50] [abbrv] hadoop git commit: HADOOP-14049. Honour AclBit flag associated to file/folder permission for Azure datalake account. Contributed by Vishwajeet Dusane

Posted by xg...@apache.org.
HADOOP-14049. Honour AclBit flag associated to file/folder permission for Azure datalake account. Contributed by Vishwajeet Dusane


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

Branch: refs/heads/YARN-5734
Commit: f4329990250bed62efdebe3ce2bc740092cf9573
Parents: a77f432
Author: Mingliang Liu <li...@apache.org>
Authored: Thu Feb 16 15:14:25 2017 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Thu Feb 16 15:14:25 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/adl/AdlFileSystem.java |  6 +++-
 .../hadoop/fs/adl/TestADLResponseData.java      | 21 +++++++++++++
 .../apache/hadoop/fs/adl/TestGetFileStatus.java | 25 +++++++++++++++
 .../apache/hadoop/fs/adl/TestListStatus.java    | 32 ++++++++++++++++++++
 4 files changed, 83 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4329990/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
index 303b7bc..fb0feda 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
@@ -593,7 +593,11 @@ public class AdlFileSystem extends FileSystem {
     boolean isDirectory = entry.type == DirectoryEntryType.DIRECTORY;
     long lastModificationData = entry.lastModifiedTime.getTime();
     long lastAccessTime = entry.lastAccessTime.getTime();
-    FsPermission permission = new AdlPermission(aclBitStatus,
+    // set aclBit from ADLS backend response if
+    // ADL_SUPPORT_ACL_BIT_IN_FSPERMISSION is true.
+    final boolean aclBit = aclBitStatus ? entry.aclBit : false;
+
+    FsPermission permission = new AdlPermission(aclBit,
         Short.valueOf(entry.permission, 8));
     String user = entry.user;
     String group = entry.group;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4329990/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestADLResponseData.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestADLResponseData.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestADLResponseData.java
index 24eb314..788242e 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestADLResponseData.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestADLResponseData.java
@@ -66,6 +66,15 @@ public final class TestADLResponseData {
         "\"owner\":\"NotSupportYet\",\"group\":\"NotSupportYet\"}}";
   }
 
+  public static String getGetFileStatusJSONResponse(boolean aclBit) {
+    return "{\"FileStatus\":{\"length\":1024," +
+        "\"pathSuffix\":\"\",\"type\":\"FILE\",\"blockSize\":268435456," +
+        "\"accessTime\":1452103827023,\"modificationTime\":1452103827023," +
+        "\"replication\":0,\"permission\":\"777\"," +
+        "\"owner\":\"NotSupportYet\",\"group\":\"NotSupportYet\",\"aclBit\":\""
+        + aclBit + "\"}}";
+  }
+
   public static String getListFileStatusJSONResponse(int dirSize) {
     String list = "";
     for (int i = 0; i < dirSize; ++i) {
@@ -81,6 +90,18 @@ public final class TestADLResponseData {
     return "{\"FileStatuses\":{\"FileStatus\":[" + list + "]}}";
   }
 
+  public static String getListFileStatusJSONResponse(boolean aclBit) {
+    return "{\"FileStatuses\":{\"FileStatus\":[{\"length\":0,\"pathSuffix\":\""
+        + java.util.UUID.randomUUID()
+        + "\",\"type\":\"DIRECTORY\",\"blockSize\":0,"
+        + "\"accessTime\":1481184513488,"
+        + "\"modificationTime\":1481184513488,\"replication\":0,"
+        + "\"permission\":\"770\","
+        + "\"owner\":\"4b27fe1a-d9ab-4a04-ad7a-4bba72cd9e6c\","
+        + "\"group\":\"4b27fe1a-d9ab-4a04-ad7a-4bba72cd9e6c\",\"aclBit\":\""
+        + aclBit + "\"}]}}";
+  }
+
   public static String getJSONResponse(boolean status) {
     return "{\"boolean\":" + status + "}";
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4329990/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
index 08c805e..78ef931 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
@@ -67,4 +67,29 @@ public class TestGetFileStatus extends AdlMockWebServer {
     Assert.assertEquals("NotSupportYet", fileStatus.getGroup());
   }
 
+    @Test
+    public void getFileStatusAclBit()
+            throws URISyntaxException, IOException {
+        // With ACLBIT set to true
+        getMockServer().enqueue(new MockResponse().setResponseCode(200)
+                .setBody(TestADLResponseData.getGetFileStatusJSONResponse(true)));
+        long startTime = Time.monotonicNow();
+        FileStatus fileStatus = getMockAdlFileSystem()
+                .getFileStatus(new Path("/test1/test2"));
+        long endTime = Time.monotonicNow();
+        LOG.debug("Time : " + (endTime - startTime));
+        Assert.assertTrue(fileStatus.isFile());
+        Assert.assertEquals(true, fileStatus.getPermission().getAclBit());
+
+        // With ACLBIT set to false
+        getMockServer().enqueue(new MockResponse().setResponseCode(200)
+                .setBody(TestADLResponseData.getGetFileStatusJSONResponse(false)));
+        startTime = Time.monotonicNow();
+        fileStatus = getMockAdlFileSystem()
+                .getFileStatus(new Path("/test1/test2"));
+        endTime = Time.monotonicNow();
+        LOG.debug("Time : " + (endTime - startTime));
+        Assert.assertTrue(fileStatus.isFile());
+        Assert.assertEquals(false, fileStatus.getPermission().getAclBit());
+    }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4329990/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
index c151e89..dac8886 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestListStatus.java
@@ -29,6 +29,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.net.URISyntaxException;
 
 /**
  * This class is responsible for testing local listStatus implementation to
@@ -100,4 +101,35 @@ public class TestListStatus extends AdlMockWebServer {
     LOG.debug("Time : " + (endTime - startTime));
   }
 
+  @Test
+  public void listStatusAclBit()
+          throws URISyntaxException, IOException {
+    // With ACLBIT set to true
+    getMockServer().enqueue(new MockResponse().setResponseCode(200)
+            .setBody(TestADLResponseData.getListFileStatusJSONResponse(true)));
+    FileStatus[] ls = null;
+    long startTime = Time.monotonicNow();
+    ls = getMockAdlFileSystem()
+            .listStatus(new Path("/test1/test2"));
+    long endTime = Time.monotonicNow();
+    LOG.debug("Time : " + (endTime - startTime));
+    for (int i = 0; i < ls.length; i++) {
+      Assert.assertTrue(ls[i].isDirectory());
+      Assert.assertEquals(true, ls[i].getPermission().getAclBit());
+    }
+
+    // With ACLBIT set to false
+    ls = null;
+    getMockServer().enqueue(new MockResponse().setResponseCode(200)
+            .setBody(TestADLResponseData.getListFileStatusJSONResponse(false)));
+    startTime = Time.monotonicNow();
+    ls = getMockAdlFileSystem()
+            .listStatus(new Path("/test1/test2"));
+    endTime = Time.monotonicNow();
+    LOG.debug("Time : " + (endTime - startTime));
+    for (int i = 0; i < ls.length; i++) {
+      Assert.assertTrue(ls[i].isDirectory());
+      Assert.assertEquals(false, ls[i].getPermission().getAclBit());
+    }
+  }
 }


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


[18/50] [abbrv] hadoop git commit: HDFS-8498. Blocks can be committed with wrong size. Contributed by Jing Zhao.

Posted by xg...@apache.org.
HDFS-8498. Blocks can be committed with wrong size. 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/627da6f7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/627da6f7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/627da6f7

Branch: refs/heads/YARN-5734
Commit: 627da6f7178e18aa41996969c408b6f344e297d1
Parents: 0fc6f38
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Feb 15 10:44:37 2017 -0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Feb 15 10:44:37 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DataStreamer.java    | 100 +++++++++++++------
 .../apache/hadoop/hdfs/StripedDataStreamer.java |   8 +-
 .../apache/hadoop/hdfs/TestDFSOutputStream.java |   3 +-
 3 files changed, 72 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/627da6f7/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index 8e6eb63..0268537 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -142,8 +142,6 @@ class DataStreamer extends Daemon {
 
     /**
      * Record a connection exception.
-     * @param e
-     * @throws InvalidEncryptionKeyException
      */
     void recordFailure(final InvalidEncryptionKeyException e)
         throws InvalidEncryptionKeyException {
@@ -178,9 +176,8 @@ class DataStreamer extends Daemon {
         final StorageType[] targetStorageTypes,
         final Token<BlockTokenIdentifier> blockToken) throws IOException {
       //send the TRANSFER_BLOCK request
-      new Sender(out)
-          .transferBlock(block, blockToken, dfsClient.clientName, targets,
-              targetStorageTypes);
+      new Sender(out).transferBlock(block.getCurrentBlock(), blockToken,
+          dfsClient.clientName, targets, targetStorageTypes);
       out.flush();
       //ack
       BlockOpResponseProto transferResponse = BlockOpResponseProto
@@ -199,6 +196,42 @@ class DataStreamer extends Daemon {
     }
   }
 
+  static class BlockToWrite {
+    private ExtendedBlock currentBlock;
+
+    BlockToWrite(ExtendedBlock block) {
+      setCurrentBlock(block);
+    }
+
+    synchronized ExtendedBlock getCurrentBlock() {
+      return currentBlock == null ? null : new ExtendedBlock(currentBlock);
+    }
+
+    synchronized long getNumBytes() {
+      return currentBlock == null ? 0 : currentBlock.getNumBytes();
+    }
+
+    synchronized void setCurrentBlock(ExtendedBlock block) {
+      currentBlock = (block == null || block.getLocalBlock() == null) ?
+          null : new ExtendedBlock(block);
+    }
+
+    synchronized void setNumBytes(long numBytes) {
+      assert currentBlock != null;
+      currentBlock.setNumBytes(numBytes);
+    }
+
+    synchronized void setGenerationStamp(long generationStamp) {
+      assert currentBlock != null;
+      currentBlock.setGenerationStamp(generationStamp);
+    }
+
+    @Override
+    public synchronized String toString() {
+      return currentBlock == null ? "null" : currentBlock.toString();
+    }
+  }
+
   /**
    * Create a socket for a write pipeline
    *
@@ -440,7 +473,7 @@ class DataStreamer extends Daemon {
   }
 
   private volatile boolean streamerClosed = false;
-  protected volatile ExtendedBlock block; // its length is number of bytes acked
+  protected final BlockToWrite block; // its length is number of bytes acked
   protected Token<BlockTokenIdentifier> accessToken;
   private DataOutputStream blockStream;
   private DataInputStream blockReplyStream;
@@ -508,7 +541,7 @@ class DataStreamer extends Daemon {
                        ByteArrayManager byteArrayManage,
                        boolean isAppend, String[] favoredNodes,
                        EnumSet<AddBlockFlag> flags) {
-    this.block = block;
+    this.block = new BlockToWrite(block);
     this.dfsClient = dfsClient;
     this.src = src;
     this.progress = progress;
@@ -1322,7 +1355,7 @@ class DataStreamer extends Daemon {
       LocatedBlock lb;
       //get a new datanode
       lb = dfsClient.namenode.getAdditionalDatanode(
-          src, stat.getFileId(), block, nodes, storageIDs,
+          src, stat.getFileId(), block.getCurrentBlock(), nodes, storageIDs,
           exclude.toArray(new DatanodeInfo[exclude.size()]),
           1, dfsClient.clientName);
       // a new node was allocated by the namenode. Update nodes.
@@ -1440,7 +1473,7 @@ class DataStreamer extends Daemon {
     } // while
 
     if (success) {
-      block = updatePipeline(newGS);
+      updatePipeline(newGS);
     }
   }
 
@@ -1536,21 +1569,22 @@ class DataStreamer extends Daemon {
   }
 
   private LocatedBlock updateBlockForPipeline() throws IOException {
-    return dfsClient.namenode.updateBlockForPipeline(block,
+    return dfsClient.namenode.updateBlockForPipeline(block.getCurrentBlock(),
         dfsClient.clientName);
   }
 
-  static ExtendedBlock newBlock(ExtendedBlock b, final long newGS) {
-    return new ExtendedBlock(b.getBlockPoolId(), b.getBlockId(),
-        b.getNumBytes(), newGS);
+  void updateBlockGS(final long newGS) {
+    block.setGenerationStamp(newGS);
   }
 
   /** update pipeline at the namenode */
-  ExtendedBlock updatePipeline(long newGS) throws IOException {
-    final ExtendedBlock newBlock = newBlock(block, newGS);
-    dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock,
-        nodes, storageIDs);
-    return newBlock;
+  private void updatePipeline(long newGS) throws IOException {
+    final ExtendedBlock oldBlock = block.getCurrentBlock();
+    // the new GS has been propagated to all DN, it should be ok to update the
+    // local block state
+    updateBlockGS(newGS);
+    dfsClient.namenode.updatePipeline(dfsClient.clientName, oldBlock,
+        block.getCurrentBlock(), nodes, storageIDs);
   }
 
   DatanodeInfo[] getExcludedNodes() {
@@ -1570,31 +1604,29 @@ class DataStreamer extends Daemon {
     StorageType[] storageTypes;
     int count = dfsClient.getConf().getNumBlockWriteRetry();
     boolean success;
-    ExtendedBlock oldBlock = block;
+    final ExtendedBlock oldBlock = block.getCurrentBlock();
     do {
       errorState.resetInternalError();
       lastException.clear();
 
       DatanodeInfo[] excluded = getExcludedNodes();
-      block = oldBlock;
-      lb = locateFollowingBlock(excluded.length > 0 ? excluded : null);
-      block = lb.getBlock();
+      lb = locateFollowingBlock(
+          excluded.length > 0 ? excluded : null, oldBlock);
+      block.setCurrentBlock(lb.getBlock());
       block.setNumBytes(0);
       bytesSent = 0;
       accessToken = lb.getBlockToken();
       nodes = lb.getLocations();
       storageTypes = lb.getStorageTypes();
 
-      //
       // Connect to first DataNode in the list.
-      //
       success = createBlockOutputStream(nodes, storageTypes, 0L, false);
 
       if (!success) {
         LOG.warn("Abandoning " + block);
-        dfsClient.namenode.abandonBlock(block, stat.getFileId(), src,
-            dfsClient.clientName);
-        block = null;
+        dfsClient.namenode.abandonBlock(block.getCurrentBlock(),
+            stat.getFileId(), src, dfsClient.clientName);
+        block.setCurrentBlock(null);
         final DatanodeInfo badNode = nodes[errorState.getBadNodeIndex()];
         LOG.warn("Excluding datanode " + badNode);
         excludedNodes.put(badNode, badNode);
@@ -1655,7 +1687,7 @@ class DataStreamer extends Daemon {
 
         // We cannot change the block length in 'block' as it counts the number
         // of bytes ack'ed.
-        ExtendedBlock blockCopy = new ExtendedBlock(block);
+        ExtendedBlock blockCopy = block.getCurrentBlock();
         blockCopy.setNumBytes(stat.getBlockSize());
 
         boolean[] targetPinnings = getPinnings(nodes);
@@ -1765,9 +1797,9 @@ class DataStreamer extends Daemon {
     }
   }
 
-  private LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
-      throws IOException {
-    return DFSOutputStream.addBlock(excludedNodes, dfsClient, src, block,
+  private LocatedBlock locateFollowingBlock(DatanodeInfo[] excluded,
+      ExtendedBlock oldBlock) throws IOException {
+    return DFSOutputStream.addBlock(excluded, dfsClient, src, oldBlock,
         stat.getFileId(), favoredNodes, addBlockFlags);
   }
 
@@ -1811,7 +1843,7 @@ class DataStreamer extends Daemon {
    * @return the block this streamer is writing to
    */
   ExtendedBlock getBlock() {
-    return block;
+    return block.getCurrentBlock();
   }
 
   /**
@@ -2016,6 +2048,8 @@ class DataStreamer extends Daemon {
 
   @Override
   public String toString() {
-    return block == null? "block==null": "" + block.getLocalBlock();
+    final ExtendedBlock extendedBlock = block.getCurrentBlock();
+    return extendedBlock == null ?
+        "block==null" : "" + extendedBlock.getLocalBlock();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/627da6f7/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
index 89ab6a3..b457edb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
@@ -71,7 +71,7 @@ public class StripedDataStreamer extends DataStreamer {
 
   @Override
   protected void endBlock() {
-    coordinator.offerEndBlock(index, block);
+    coordinator.offerEndBlock(index, block.getCurrentBlock());
     super.endBlock();
   }
 
@@ -93,7 +93,7 @@ public class StripedDataStreamer extends DataStreamer {
   protected LocatedBlock nextBlockOutputStream() throws IOException {
     boolean success;
     LocatedBlock lb = getFollowingBlock();
-    block = lb.getBlock();
+    block.setCurrentBlock(lb.getBlock());
     block.setNumBytes(0);
     bytesSent = 0;
     accessToken = lb.getBlockToken();
@@ -105,7 +105,7 @@ public class StripedDataStreamer extends DataStreamer {
     success = createBlockOutputStream(nodes, storageTypes, 0L, false);
 
     if (!success) {
-      block = null;
+      block.setCurrentBlock(null);
       final DatanodeInfo badNode = nodes[getErrorState().getBadNodeIndex()];
       LOG.warn("Excluding datanode " + badNode);
       excludedNodes.put(badNode, badNode);
@@ -161,7 +161,7 @@ public class StripedDataStreamer extends DataStreamer {
         success = coordinator.takeStreamerUpdateResult(index);
         if (success) {
           // if all succeeded, update its block using the new GS
-          block = newBlock(block, newGS);
+          updateBlockGS(newGS);
         } else {
           // otherwise close the block stream and restart the recovery process
           closeStream();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/627da6f7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
index 750103d..9ec01b6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
@@ -110,8 +110,7 @@ public class TestDFSOutputStream {
    * packet size < 64kB. See HDFS-7308 for details.
    */
   @Test
-  public void testComputePacketChunkSize()
-      throws Exception {
+  public void testComputePacketChunkSize() throws Exception {
     DistributedFileSystem fs = cluster.getFileSystem();
     FSDataOutputStream os = fs.create(new Path("/test"));
     DFSOutputStream dos = (DFSOutputStream) Whitebox.getInternalState(os,


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


[45/50] [abbrv] hadoop git commit: HADOOP-13826. S3A Deadlock in multipart copy due to thread pool limits. Contributed by Sean Mackrory.

Posted by xg...@apache.org.
HADOOP-13826. S3A Deadlock in multipart copy due to thread pool limits. Contributed by  Sean Mackrory.

(cherry picked from commit e3a74e0369e6e2217d1280179b390227fe1b1684)


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

Branch: refs/heads/YARN-5734
Commit: 2158496f6bed5f9d14751b82bd5d43b9fd786b95
Parents: a07ddef
Author: Steve Loughran <st...@apache.org>
Authored: Tue Feb 21 17:54:43 2017 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Feb 21 18:28:49 2017 +0000

----------------------------------------------------------------------
 .../s3a/BlockingThreadPoolExecutorService.java  |   2 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |  21 ++-
 .../fs/s3a/scale/ITestS3AConcurrentOps.java     | 167 +++++++++++++++++++
 3 files changed, 184 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2158496f/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java
index 5ff96a5..5b25730 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BlockingThreadPoolExecutorService.java
@@ -86,7 +86,7 @@ final class BlockingThreadPoolExecutorService
    * @return a thread factory that creates named, daemon threads with
    * the supplied exception handler and normal priority
    */
-  private static ThreadFactory newDaemonThreadFactory(final String prefix) {
+  static ThreadFactory newDaemonThreadFactory(final String prefix) {
     final ThreadFactory namedFactory = getNamedThreadFactory(prefix);
     return new ThreadFactory() {
       @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2158496f/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index bffc210..8b1a6d0 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -29,7 +29,10 @@ import java.util.Date;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ExecutorService;
 import java.util.Objects;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -131,7 +134,8 @@ public class S3AFileSystem extends FileSystem {
   private long partSize;
   private boolean enableMultiObjectsDelete;
   private TransferManager transfers;
-  private ListeningExecutorService threadPoolExecutor;
+  private ListeningExecutorService boundedThreadPool;
+  private ExecutorService unboundedThreadPool;
   private long multiPartThreshold;
   public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class);
   private static final Logger PROGRESS =
@@ -216,11 +220,17 @@ public class S3AFileSystem extends FileSystem {
           MAX_TOTAL_TASKS, DEFAULT_MAX_TOTAL_TASKS, 1);
       long keepAliveTime = longOption(conf, KEEPALIVE_TIME,
           DEFAULT_KEEPALIVE_TIME, 0);
-      threadPoolExecutor = BlockingThreadPoolExecutorService.newInstance(
+      boundedThreadPool = BlockingThreadPoolExecutorService.newInstance(
           maxThreads,
           maxThreads + totalTasks,
           keepAliveTime, TimeUnit.SECONDS,
           "s3a-transfer-shared");
+      unboundedThreadPool = new ThreadPoolExecutor(
+          maxThreads, Integer.MAX_VALUE,
+          keepAliveTime, TimeUnit.SECONDS,
+          new LinkedBlockingQueue<Runnable>(),
+          BlockingThreadPoolExecutorService.newDaemonThreadFactory(
+              "s3a-transfer-unbounded"));
 
       initTransferManager();
 
@@ -307,7 +317,7 @@ public class S3AFileSystem extends FileSystem {
     transferConfiguration.setMultipartCopyPartSize(partSize);
     transferConfiguration.setMultipartCopyThreshold(multiPartThreshold);
 
-    transfers = new TransferManager(s3, threadPoolExecutor);
+    transfers = new TransferManager(s3, unboundedThreadPool);
     transfers.setConfiguration(transferConfiguration);
   }
 
@@ -585,7 +595,7 @@ public class S3AFileSystem extends FileSystem {
       output = new FSDataOutputStream(
           new S3ABlockOutputStream(this,
               key,
-              new SemaphoredDelegatingExecutor(threadPoolExecutor,
+              new SemaphoredDelegatingExecutor(boundedThreadPool,
                   blockOutputActiveBlocks, true),
               progress,
               partSize,
@@ -2060,7 +2070,8 @@ public class S3AFileSystem extends FileSystem {
     if (blockFactory != null) {
       sb.append(", blockFactory=").append(blockFactory);
     }
-    sb.append(", executor=").append(threadPoolExecutor);
+    sb.append(", boundedExecutor=").append(boundedThreadPool);
+    sb.append(", unboundedExecutor=").append(unboundedThreadPool);
     sb.append(", statistics {")
         .append(statistics)
         .append("}");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2158496f/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java
new file mode 100644
index 0000000..b4d3862
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.scale;
+
+import java.io.IOException;
+
+import java.net.URI;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.contract.ContractTestUtils.NanoTimer;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+
+import org.junit.After;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
+/**
+ * Tests concurrent operations on a single S3AFileSystem instance.
+ */
+public class ITestS3AConcurrentOps extends S3AScaleTestBase {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      ITestS3AConcurrentOps.class);
+  private final int concurrentRenames = 10;
+  private Path testRoot;
+  private Path[] source = new Path[concurrentRenames];
+  private Path[] target = new Path[concurrentRenames];
+  private S3AFileSystem fs;
+  private S3AFileSystem auxFs;
+
+  @Override
+  protected int getTestTimeoutSeconds() {
+    return 16 * 60;
+  }
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    fs = getRestrictedFileSystem();
+    auxFs = getNormalFileSystem();
+
+    testRoot = path("/ITestS3AConcurrentOps");
+    testRoot = S3ATestUtils.createTestPath(testRoot);
+
+    for (int i = 0; i < concurrentRenames; i++){
+      source[i] = new Path(testRoot, "source" + i);
+      target[i] = new Path(testRoot, "target" + i);
+    }
+
+    LOG.info("Generating data...");
+    auxFs.mkdirs(testRoot);
+    byte[] zeroes = ContractTestUtils.dataset(1024*1024, 0, Integer.MAX_VALUE);
+    for (Path aSource : source) {
+      try(FSDataOutputStream out = auxFs.create(aSource)) {
+        for (int mb = 0; mb < 20; mb++) {
+          LOG.debug("{}: Block {}...", aSource, mb);
+          out.write(zeroes);
+        }
+      }
+    }
+    LOG.info("Data generated...");
+  }
+
+  private S3AFileSystem getRestrictedFileSystem() throws Exception {
+    Configuration conf = getConfiguration();
+    conf.setInt(MAX_THREADS, 2);
+    conf.setInt(MAX_TOTAL_TASKS, 1);
+
+    conf.set(MIN_MULTIPART_THRESHOLD, "10M");
+    conf.set(MULTIPART_SIZE, "5M");
+
+    S3AFileSystem s3a = getFileSystem();
+    URI rootURI = new URI(conf.get(TEST_FS_S3A_NAME));
+    s3a.initialize(rootURI, conf);
+    return s3a;
+  }
+
+  private S3AFileSystem getNormalFileSystem() throws Exception {
+    S3AFileSystem s3a = new S3AFileSystem();
+    Configuration conf = new Configuration();
+    URI rootURI = new URI(conf.get(TEST_FS_S3A_NAME));
+    s3a.initialize(rootURI, conf);
+    return s3a;
+  }
+
+  @After
+  public void teardown() throws Exception {
+    if (auxFs != null) {
+      auxFs.delete(testRoot, true);
+    }
+  }
+
+  /**
+   * Attempts to trigger a deadlock that would happen if any bounded resource
+   * pool became saturated with control tasks that depended on other tasks
+   * that now can't enter the resource pool to get completed.
+   */
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testParallelRename() throws InterruptedException,
+      ExecutionException, IOException {
+    ExecutorService executor = Executors.newFixedThreadPool(
+        concurrentRenames, new ThreadFactory() {
+          private AtomicInteger count = new AtomicInteger(0);
+
+          public Thread newThread(Runnable r) {
+            return new Thread(r,
+                "testParallelRename" + count.getAndIncrement());
+          }
+        });
+    ((ThreadPoolExecutor)executor).prestartAllCoreThreads();
+    Future<Boolean>[] futures = new Future[concurrentRenames];
+    for (int i = 0; i < concurrentRenames; i++) {
+      final int index = i;
+      futures[i] = executor.submit(new Callable<Boolean>() {
+        @Override
+        public Boolean call() throws Exception {
+          NanoTimer timer = new NanoTimer();
+          boolean result = fs.rename(source[index], target[index]);
+          timer.end("parallel rename %d", index);
+          LOG.info("Rename {} ran from {} to {}", index,
+              timer.getStartTime(), timer.getEndTime());
+          return result;
+        }
+      });
+    }
+    LOG.info("Waiting for tasks to complete...");
+    LOG.info("Deadlock may have occurred if nothing else is logged" +
+        " or the test times out");
+    for (int i = 0; i < concurrentRenames; i++) {
+      assertTrue("No future " + i, futures[i].get());
+      assertPathExists("target path", target[i]);
+      assertPathDoesNotExist("source path", source[i]);
+    }
+    LOG.info("All tasks have completed successfully");
+  }
+}


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


[24/50] [abbrv] hadoop git commit: YARN-6200. Reverting since the same functionality achieved by YARN-1623. Revert "YARN-5068. Expose scheduler queue to application master. (Harish Jaiprakash via rohithsharmaks)"

Posted by xg...@apache.org.
YARN-6200. Reverting since the same functionality achieved by YARN-1623.
Revert "YARN-5068. Expose scheduler queue to application master. (Harish Jaiprakash via rohithsharmaks)"

This reverts commit b7ac85259c7d20c33bef9c9cb40b8aabcab70755.


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

Branch: refs/heads/YARN-5734
Commit: a393e84c6f57d50a471fc902dfd07ca1b4128a0e
Parents: 74dd142
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Thu Feb 16 18:00:25 2017 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Thu Feb 16 18:00:25 2017 +0530

----------------------------------------------------------------------
 .../apache/hadoop/yarn/api/ApplicationConstants.java  |  7 -------
 .../server/resourcemanager/amlauncher/AMLauncher.java | 14 --------------
 .../TestApplicationMasterLauncher.java                |  5 -----
 3 files changed, 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a393e84c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java
index 760e251..64bcc44 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java
@@ -160,13 +160,6 @@ public interface ApplicationConstants {
     LD_LIBRARY_PATH("LD_LIBRARY_PATH"),
 
     /**
-     * $YARN_RESOURCEMANAGER_APPLICATION_QUEUE
-     * The queue into which the app was submitted/launched.
-     */
-    YARN_RESOURCEMANAGER_APPLICATION_QUEUE(
-        "YARN_RESOURCEMANAGER_APPLICATION_QUEUE"),
-
-    /**
      * $HADOOP_CONF_DIR
      * Final, non-modifiable.
      */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a393e84c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.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/amlauncher/AMLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
index d33360b..05f9f47 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
@@ -51,7 +51,6 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.client.NMProxy;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
@@ -189,9 +188,6 @@ public class AMLauncher implements Runnable {
     ContainerLaunchContext container =
         applicationMasterContext.getAMContainerSpec();
 
-    // Populate the current queue name in the environment variable.
-    setupQueueNameEnv(container, applicationMasterContext);
-
     // Finalize the container
     setupTokens(container, containerID);
     // set the flow context optionally for timeline service v.2
@@ -200,16 +196,6 @@ public class AMLauncher implements Runnable {
     return container;
   }
 
-  private void setupQueueNameEnv(ContainerLaunchContext container,
-      ApplicationSubmissionContext applicationMasterContext) {
-    String queueName = applicationMasterContext.getQueue();
-    if (queueName == null) {
-      queueName = YarnConfiguration.DEFAULT_QUEUE_NAME;
-    }
-    container.getEnvironment().put(ApplicationConstants.Environment
-            .YARN_RESOURCEMANAGER_APPLICATION_QUEUE.key(), queueName);
-  }
-
   @Private
   @VisibleForTesting
   protected void setupTokens(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a393e84c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.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/TestApplicationMasterLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
index 9a46e01..08b180f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
@@ -101,7 +101,6 @@ public class TestApplicationMasterLauncher {
     String nmHostAtContainerManager = null;
     long submitTimeAtContainerManager;
     int maxAppAttempts;
-    private String queueName;
 
     @Override
     public StartContainersResponse
@@ -130,8 +129,6 @@ public class TestApplicationMasterLauncher {
       submitTimeAtContainerManager =
           Long.parseLong(env.get(ApplicationConstants.APP_SUBMIT_TIME_ENV));
       maxAppAttempts = YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS;
-      queueName = env.get(ApplicationConstants.Environment
-              .YARN_RESOURCEMANAGER_APPLICATION_QUEUE.key());
       return StartContainersResponse.newInstance(
         new HashMap<String, ByteBuffer>(), new ArrayList<ContainerId>(),
         new HashMap<ContainerId, SerializedException>());
@@ -231,8 +228,6 @@ public class TestApplicationMasterLauncher {
       containerManager.nmHostAtContainerManager);
     Assert.assertEquals(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS,
         containerManager.maxAppAttempts);
-    Assert.assertEquals(YarnConfiguration.DEFAULT_QUEUE_NAME,
-        containerManager.queueName);
 
     MockAM am = new MockAM(rm.getRMContext(), rm
         .getApplicationMasterService(), appAttemptId);


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


[50/50] [abbrv] hadoop git commit: YARN-5951. Changes to allow CapacityScheduler to use configuration store

Posted by xg...@apache.org.
YARN-5951. Changes to allow CapacityScheduler to use configuration store


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

Branch: refs/heads/YARN-5734
Commit: 6e1a54403e8e4d43ce96c9c0cda4cb465ac8e2f3
Parents: 003ae00
Author: Jonathan Hung <jh...@linkedin.com>
Authored: Mon Jan 30 19:03:48 2017 -0800
Committer: Xuan <xg...@apache.org>
Committed: Tue Feb 21 15:56:59 2017 -0800

----------------------------------------------------------------------
 .../scheduler/capacity/CapacityScheduler.java   | 36 +++++------
 .../CapacitySchedulerConfiguration.java         | 10 +++
 .../capacity/conf/CSConfigurationProvider.java  | 46 ++++++++++++++
 .../conf/FileBasedCSConfigurationProvider.java  | 67 ++++++++++++++++++++
 .../scheduler/capacity/conf/package-info.java   | 29 +++++++++
 .../capacity/TestCapacityScheduler.java         |  4 +-
 6 files changed, 170 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e1a5440/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
index 3517764..de0f162 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import java.io.IOException;
-import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -108,6 +107,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.Activi
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityDiagnosticConstant;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.AllocationState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.CSConfigurationProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.FileBasedCSConfigurationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.AssignmentInformation;
@@ -166,6 +167,7 @@ public class CapacityScheduler extends
 
   private int offswitchPerHeartbeatLimit;
 
+  private CSConfigurationProvider csConfProvider;
 
   @Override
   public void setConf(Configuration conf) {
@@ -289,7 +291,18 @@ public class CapacityScheduler extends
       IOException {
     try {
       writeLock.lock();
-      this.conf = loadCapacitySchedulerConfiguration(configuration);
+      String confProviderStr = configuration.get(
+          CapacitySchedulerConfiguration.CS_CONF_PROVIDER,
+          CapacitySchedulerConfiguration.DEFAULT_CS_CONF_PROVIDER);
+      if (confProviderStr.equals(
+          CapacitySchedulerConfiguration.FILE_CS_CONF_PROVIDER)) {
+        this.csConfProvider = new FileBasedCSConfigurationProvider(rmContext);
+      } else {
+        throw new IOException("Invalid CS configuration provider: " +
+            confProviderStr);
+      }
+      this.csConfProvider.init(configuration);
+      this.conf = this.csConfProvider.loadConfiguration(configuration);
       validateConf(this.conf);
       this.minimumAllocation = this.conf.getMinimumAllocation();
       initMaximumResourceCapability(this.conf.getMaximumAllocation());
@@ -396,7 +409,7 @@ public class CapacityScheduler extends
       writeLock.lock();
       Configuration configuration = new Configuration(newConf);
       CapacitySchedulerConfiguration oldConf = this.conf;
-      this.conf = loadCapacitySchedulerConfiguration(configuration);
+      this.conf = csConfProvider.loadConfiguration(configuration);
       validateConf(this.conf);
       try {
         LOG.info("Re-initializing queues...");
@@ -1855,23 +1868,6 @@ public class CapacityScheduler extends
     return true;
   }
 
-  private CapacitySchedulerConfiguration loadCapacitySchedulerConfiguration(
-      Configuration configuration) throws IOException {
-    try {
-      InputStream CSInputStream =
-          this.rmContext.getConfigurationProvider()
-              .getConfigurationInputStream(configuration,
-                  YarnConfiguration.CS_CONFIGURATION_FILE);
-      if (CSInputStream != null) {
-        configuration.addResource(CSInputStream);
-        return new CapacitySchedulerConfiguration(configuration, false);
-      }
-      return new CapacitySchedulerConfiguration(configuration, true);
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-  }
-
   private String getDefaultReservationQueueName(String planQueueName) {
     return planQueueName + ReservationConstants.DEFAULT_QUEUE_SUFFIX;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e1a5440/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
index 43ec390..d11b5ac 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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
@@ -285,6 +285,16 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   @Private
   public static final boolean DEFAULT_LAZY_PREEMPTION_ENABLED = false;
 
+  @Private
+  public static final String CS_CONF_PROVIDER = PREFIX
+      + "configuration.provider";
+
+  @Private
+  public static final String FILE_CS_CONF_PROVIDER = "file";
+
+  @Private
+  public static final String DEFAULT_CS_CONF_PROVIDER = FILE_CS_CONF_PROVIDER;
+
   AppPriorityACLConfigurationParser priorityACLConfig = new AppPriorityACLConfigurationParser();
 
   public CapacitySchedulerConfiguration() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e1a5440/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/CSConfigurationProvider.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/conf/CSConfigurationProvider.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/conf/CSConfigurationProvider.java
new file mode 100644
index 0000000..c9984ac
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/CSConfigurationProvider.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+
+import java.io.IOException;
+
+/**
+ * Configuration provider for {@link CapacityScheduler}.
+ */
+public interface CSConfigurationProvider {
+
+  /**
+   * Initialize the configuration provider with given conf.
+   * @param conf configuration to initialize with
+   */
+  void init(Configuration conf);
+
+  /**
+   * Loads capacity scheduler configuration object.
+   * @param conf initial bootstrap configuration
+   * @return CS configuration
+   * @throws IOException if fail to retrieve configuration
+   */
+  CapacitySchedulerConfiguration loadConfiguration(Configuration conf)
+      throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e1a5440/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/FileBasedCSConfigurationProvider.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/conf/FileBasedCSConfigurationProvider.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/conf/FileBasedCSConfigurationProvider.java
new file mode 100644
index 0000000..51c64fa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/FileBasedCSConfigurationProvider.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * {@link CapacityScheduler} configuration provider based on local
+ * {@code capacity-scheduler.xml} file.
+ */
+public class FileBasedCSConfigurationProvider implements
+    CSConfigurationProvider {
+
+  private RMContext rmContext;
+
+  /**
+   * Construct file based CS configuration provider with given context.
+   * @param rmContext the RM context
+   */
+  public FileBasedCSConfigurationProvider(RMContext rmContext) {
+    this.rmContext = rmContext;
+  }
+
+  @Override
+  public void init(Configuration conf) {}
+
+  @Override
+  public CapacitySchedulerConfiguration loadConfiguration(Configuration conf)
+      throws IOException {
+    try {
+      InputStream csInputStream =
+          this.rmContext.getConfigurationProvider()
+              .getConfigurationInputStream(conf,
+                  YarnConfiguration.CS_CONFIGURATION_FILE);
+      if (csInputStream != null) {
+        conf.addResource(csInputStream);
+        return new CapacitySchedulerConfiguration(conf, false);
+      }
+      return new CapacitySchedulerConfiguration(conf, true);
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e1a5440/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/package-info.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/conf/package-info.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/conf/package-info.java
new file mode 100644
index 0000000..08d0522
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/package-info.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Package
+ * org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf
+ * contains classes related to capacity scheduler configuration management.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e1a5440/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 2b60ecf..9a99cab 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
@@ -223,13 +223,13 @@ public class TestCapacityScheduler {
 
   @Test (timeout = 30000)
   public void testConfValidation() throws Exception {
-    ResourceScheduler scheduler = new CapacityScheduler();
+    CapacityScheduler scheduler = new CapacityScheduler();
     scheduler.setRMContext(resourceManager.getRMContext());
     Configuration conf = new YarnConfiguration();
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 2048);
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 1024);
     try {
-      scheduler.reinitialize(conf, mockContext);
+      scheduler.init(conf);
       fail("Exception is expected because the min memory allocation is" +
         " larger than the max memory allocation.");
     } catch (YarnRuntimeException e) {


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


[30/50] [abbrv] hadoop git commit: YARN-6171. ConcurrentModificationException on FSAppAttempt.containersToPreempt. (Miklos Szegedi via kasha)

Posted by xg...@apache.org.
YARN-6171. ConcurrentModificationException on FSAppAttempt.containersToPreempt. (Miklos Szegedi via kasha)


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

Branch: refs/heads/YARN-5734
Commit: a77f432449aad67da31bd8bf8644b71def741bde
Parents: 5d339c4
Author: Karthik Kambatla <ka...@cloudera.com>
Authored: Thu Feb 16 14:54:51 2017 -0800
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Thu Feb 16 14:54:58 2017 -0800

----------------------------------------------------------------------
 .../scheduler/fair/FSAppAttempt.java            | 49 +++++++++++---------
 .../scheduler/fair/FairScheduler.java           | 15 +++---
 2 files changed, 34 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a77f4324/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
index 563b892..b1bb9a0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
@@ -83,8 +83,10 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
   private Resource fairShare = Resources.createResource(0, 0);
 
   // Preemption related variables
+  private final Object preemptionVariablesLock = new Object();
   private final Resource preemptedResources = Resources.clone(Resources.none());
   private final Set<RMContainer> containersToPreempt = new HashSet<>();
+
   private Resource fairshareStarvation = Resources.none();
   private long lastTimeAtFairShare;
   private long nextStarvationCheck;
@@ -552,29 +554,29 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
   }
 
   void trackContainerForPreemption(RMContainer container) {
-    if (containersToPreempt.add(container)) {
-      synchronized (preemptedResources) {
+    synchronized (preemptionVariablesLock) {
+      if (containersToPreempt.add(container)) {
         Resources.addTo(preemptedResources, container.getAllocatedResource());
       }
     }
   }
 
   private void untrackContainerForPreemption(RMContainer container) {
-    if (containersToPreempt.remove(container)) {
-      synchronized (preemptedResources) {
+    synchronized (preemptionVariablesLock) {
+      if (containersToPreempt.remove(container)) {
         Resources.subtractFrom(preemptedResources,
             container.getAllocatedResource());
       }
     }
   }
 
-  Set<RMContainer> getPreemptionContainers() {
-    return containersToPreempt;
-  }
-
-  private Resource getPreemptedResources() {
-    synchronized (preemptedResources) {
-      return preemptedResources;
+  Set<ContainerId> getPreemptionContainerIds() {
+    synchronized (preemptionVariablesLock) {
+      Set<ContainerId> preemptionContainerIds = new HashSet<>();
+      for (RMContainer container : containersToPreempt) {
+        preemptionContainerIds.add(container.getContainerId());
+      }
+      return preemptionContainerIds;
     }
   }
 
@@ -591,9 +593,11 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
       return false;
     }
 
-    if (containersToPreempt.contains(container)) {
-      // The container is already under consideration for preemption
-      return false;
+    synchronized (preemptionVariablesLock) {
+      if (containersToPreempt.contains(container)) {
+        // The container is already under consideration for preemption
+        return false;
+      }
     }
 
     // Check if the app's allocation will be over its fairshare even
@@ -969,7 +973,8 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
             if (LOG.isTraceEnabled()) {
               LOG.trace("Assign container on " + node.getNodeName()
                   + " node, assignType: OFF_SWITCH" + ", allowedLocality: "
-                  + allowedLocality + ", priority: " + schedulerKey.getPriority()
+                  + allowedLocality + ", priority: "
+                  + schedulerKey.getPriority()
                   + ", app attempt id: " + this.attemptId);
             }
             return assignContainer(node, offswitchAsk, NodeType.OFF_SWITCH,
@@ -1226,13 +1231,13 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
 
   @Override
   public Resource getResourceUsage() {
-    /*
-     * getResourcesToPreempt() returns zero, except when there are containers
-     * to preempt. Avoid creating an object in the common case.
-     */
-    return getPreemptedResources().equals(Resources.none())
-        ? getCurrentConsumption()
-        : Resources.subtract(getCurrentConsumption(), getPreemptedResources());
+    // Subtract copies the object, so that we have a snapshot,
+    // in case usage changes, while the caller is using the value
+    synchronized (preemptionVariablesLock) {
+      return containersToPreempt.isEmpty()
+          ? getCurrentConsumption()
+          : Resources.subtract(getCurrentConsumption(), preemptedResources);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a77f4324/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index c5bf02a..a15e6b5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -103,6 +103,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 /**
  * A scheduler that schedules resources between a set of queues. The scheduler
@@ -831,8 +832,9 @@ public class FairScheduler extends
     // Release containers
     releaseContainers(release, application);
 
+    ReentrantReadWriteLock.WriteLock lock = application.getWriteLock();
+    lock.lock();
     try {
-      application.getWriteLock().lock();
       if (!ask.isEmpty()) {
         if (LOG.isDebugEnabled()) {
           LOG.debug(
@@ -847,24 +849,21 @@ public class FairScheduler extends
         application.showRequests();
       }
     } finally {
-      application.getWriteLock().unlock();
+      lock.unlock();
     }
 
+    Set<ContainerId> preemptionContainerIds =
+        application.getPreemptionContainerIds();
     if (LOG.isDebugEnabled()) {
       LOG.debug(
           "allocate: post-update" + " applicationAttemptId=" + appAttemptId
               + " #ask=" + ask.size() + " reservation= " + application
               .getCurrentReservation());
 
-      LOG.debug("Preempting " + application.getPreemptionContainers().size()
+      LOG.debug("Preempting " + preemptionContainerIds.size()
           + " container(s)");
     }
 
-    Set<ContainerId> preemptionContainerIds = new HashSet<ContainerId>();
-    for (RMContainer container : application.getPreemptionContainers()) {
-      preemptionContainerIds.add(container.getContainerId());
-    }
-
     application.updateBlacklist(blacklistAdditions, blacklistRemovals);
 
     List<Container> newlyAllocatedContainers =


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


[09/50] [abbrv] hadoop git commit: YARN-4753. Use doxia macro to generate in-page TOC of YARN site documentation. (iwasakims)

Posted by xg...@apache.org.
YARN-4753. Use doxia macro to generate in-page TOC of YARN site documentation. (iwasakims)


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

Branch: refs/heads/YARN-5734
Commit: fbc0c2bd763e3a3aad914eb9d60b05ad4ab2825f
Parents: 353a9b2
Author: Masatake Iwasaki <iw...@apache.org>
Authored: Wed Feb 15 13:09:10 2017 +0900
Committer: Masatake Iwasaki <iw...@apache.org>
Committed: Wed Feb 15 13:09:10 2017 +0900

----------------------------------------------------------------------
 hadoop-project/src/site/site.xml                |   2 +-
 .../src/site/markdown/CapacityScheduler.md      |  14 +--
 .../src/site/markdown/DockerContainers.md       |   8 +-
 .../src/site/markdown/FairScheduler.md          |  14 +--
 .../src/site/markdown/NodeLabel.md              |  14 +--
 .../src/site/markdown/NodeManager.md            |   8 +-
 .../src/site/markdown/NodeManagerCgroups.md     |   3 +-
 .../src/site/markdown/NodeManagerRest.md        |   8 +-
 .../site/markdown/OpportunisticContainers.md    |  17 +--
 .../src/site/markdown/ReservationSystem.md      |   6 +-
 .../src/site/markdown/ResourceManagerHA.md      |  10 +-
 .../src/site/markdown/ResourceManagerRest.md    |  29 +----
 .../src/site/markdown/ResourceManagerRestart.md |  13 +--
 .../src/site/markdown/SecureContainer.md        |   2 +-
 .../src/site/markdown/TimelineServer.md         |  16 +--
 .../src/site/markdown/TimelineServiceV2.md      | 110 ++++++++-----------
 .../src/site/markdown/WebApplicationProxy.md    |   7 +-
 .../src/site/markdown/WebServicesIntro.md       |  13 +--
 .../site/markdown/WritingYarnApplications.md    |  14 +--
 .../site/markdown/YarnApplicationSecurity.md    |   2 +
 .../src/site/markdown/YarnCommands.md           |  26 +----
 21 files changed, 69 insertions(+), 267 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-project/src/site/site.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index 618ad4c..ae3aef5 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -145,7 +145,7 @@
       <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="hadoop-yarn/hadoop-yarn-site/TimelineServer.html#Timeline_Server_REST_API_v1"/>
-      <item name="Timeline Service V.2" href="hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html#Timeline_Service_REST_API_v2"/>
+      <item name="Timeline Service V.2" href="hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html#Timeline_Service_v.2_REST_API"/>
     </menu>
     
     <menu name="Hadoop Compatible File Systems" inherit="top">

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
index 9c9b03e..737bdc2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
@@ -15,19 +15,7 @@
 Hadoop: Capacity Scheduler
 ==========================
 
-* [Purpose](#Purpose)
-* [Overview](#Overview)
-* [Features](#Features)
-* [Configuration](#Configuration)
-    * [Setting up `ResourceManager` to use `CapacityScheduler`](#Setting_up_ResourceManager_to_use_CapacityScheduler`)
-    * [Setting up queues](#Setting_up_queues)
-    * [Queue Properties](#Queue_Properties)
-    * [Setup for application priority](#Setup_for_application_priority.)
-    * [Capacity Scheduler container preemption](#Capacity_Scheduler_container_preemption)
-    * [Configuring `ReservationSystem` with `CapacityScheduler`](#Configuring_ReservationSystem_with_CapacityScheduler)
-    * [Other Properties](#Other_Properties)
-    * [Reviewing the configuration of the CapacityScheduler](#Reviewing_the_configuration_of_the_CapacityScheduler)
-* [Changing Queue Configuration](#Changing_Queue_Configuration)
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
 
 Purpose
 -------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md
index b74fa7b..e66d079 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md
@@ -15,13 +15,7 @@
 Launching Applications Using Docker Containers
 ==============================================
 
-* [Overview](#Overview)
-* [Cluster Configuration](#Cluster_Configuration)
-* [Docker Image Requirements](#Docker_Image_Requirements)
-* [Application Submission](#Application_Submission)
-* [Connecting to a Secure Docker Repository](#Connecting_to_a_Secure_Docker_Repository)
-* [Example: MapReduce](#Example:_MapReduce)
-* [Example: Spark](#Example:_Spark)
+<!-- MACRO{toc|fromDepth=0|toDepth=1} -->
 
 Overview
 --------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md
index 5d7b3dc..2f0164d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/FairScheduler.md
@@ -15,19 +15,7 @@
 Hadoop: Fair Scheduler
 ======================
 
-* [Purpose](#Purpose)
-* [Introduction](#Introduction)
-* [Hierarchical queues with pluggable policies](#Hierarchical_queues_with_pluggable_policies)
-* [Automatically placing applications in queues](#Automatically_placing_applications_in_queues)
-* [Installation](#Installation)
-* [Configuration](#Configuration)
-    * [Properties that can be placed in yarn-site.xml](#Properties_that_can_be_placed_in_yarn-site.xml)
-    * [Allocation file format](#Allocation_file_format)
-    * [Queue Access Control Lists](#Queue_Access_Control_Lists)
-* [Administration](#Administration)
-    * [Modifying configuration at runtime](#Modifying_configuration_at_runtime)
-    * [Monitoring through web UI](#Monitoring_through_web_UI)
-    * [Moving applications between queues](#Moving_applications_between_queues)
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
 
 ##Purpose
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeLabel.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeLabel.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeLabel.md
index 73c9f41..39d8311 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeLabel.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeLabel.md
@@ -15,19 +15,7 @@
 YARN Node Labels
 ===============
 
-* [Overview](#Overview)
-* [Features](#Features)
-* [Configuration](#Configuration)
-    * [Setting up ResourceManager to enable Node Labels](#Setting_up_ResourceManager_to_enable_Node_Labels)
-    * [Add/modify node labels list to YARN](#Add/modify_node_labels_list_to_YARN)
-    * [Remove node labels from YARN](#Remove_node_labels_from_YARN)
-    * [Add/modify node-to-labels mapping to YARN](#Add/modify_node-to-labels_mapping_to_YARN)
-    * [Configuration of Schedulers for node labels](#Configuration_of_Schedulers_for_node_labels)
-* [Specifying node label for application](#Specifying_node_label_for_application)
-* [Monitoring](#Monitoring)
-    * [Monitoring through web UI](#Monitoring_through_web_UI)
-    * [Monitoring through commandline](#Monitoring_through_commandline)
-* [Useful links](#Useful_links)
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
 
 Overview
 --------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManager.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManager.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManager.md
index b3a92dc..3261cd7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManager.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManager.md
@@ -15,13 +15,7 @@
 NodeManager
 ===========
 
-* [Overview](#Overview)
-* [Health Checker Service](#Health_checker_service)
-    * [Disk Checker](#Disk_Checker)
-    * [External Health Script](#External_Health_Script)
-* [NodeManager Restart](#NodeManager_Restart)
-    * [Introduction](#Introduction)
-    * [Enabling NM Restart](#Enabling_NM_Restart)
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
 
 Overview
 --------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerCgroups.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerCgroups.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerCgroups.md
index 50f2faf..2704f10 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerCgroups.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerCgroups.md
@@ -15,8 +15,7 @@
 Using CGroups with YARN
 =======================
 
-* [CGroups Configuration](#CGroups_configuration)
-* [CGroups and Security](#CGroups_and_security)
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
 
 CGroups is a mechanism for aggregating/partitioning sets of tasks, and all their future children, into hierarchical groups with specialized behaviour. CGroups is a Linux kernel feature and was merged into kernel version 2.6.24. From a YARN perspective, this allows containers to be limited in their resource usage. A good example of this is CPU usage. Without CGroups, it becomes hard to limit container CPU usage. Currently, CGroups is only used for limiting CPU usage.
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerRest.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerRest.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerRest.md
index 26b21be..98d570c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerRest.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeManagerRest.md
@@ -15,13 +15,7 @@
 NodeManager REST API's
 =======================
 
-* [Overview](#Overview)
-* [Enabling CORS support](#Enabling_CORS_support)
-* [NodeManager Information API](#NodeManager_Information_API)
-* [Applications API](#Applications_API)
-* [Application API](#Application_API)
-* [Containers API](#Containers_API)
-* [Container API](#Container_API)
+<!-- MACRO{toc|fromDepth=0|toDepth=1} -->
 
 Overview
 --------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md
index 496777a..83beb07 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md
@@ -15,22 +15,7 @@
 Opportunistic Containers
 ========================
 
-* [Purpose](#Purpose)
-* [Quick Guide](#Quick_Guide)
-    * [Main Goal](#Main_Goal)
-    * [Enabling Opportunistic Containers](#Enabling_Opportunistic_Containers)
-    * [Running a Sample Job](#Running_a_Sample_Job)
-    * [Opportunistic Containers in Web UI](#Opportunistic_Containers_in_Web_UI)
-* [Overview](#Overview)
-* [Container Execution Types](#Container_Execution_Types)
-* [Execution of Opportunistic Containers](#Execution_of_Opportunistic_Containers)
-* [Allocation of Opportunistic Containers](#Allocation_of_Opportunistic_Containers)
-    * [Centralized Allocation](#Centralized_Allocation)
-    * [Distributed Allocation](#Distributed_Allocation)
-    * [Determining Nodes for Allocation](#Determining_Nodes_for_Allocation)
-    * [Rebalancing Node Load](#Rebalancing_Node_Load)
-* [Advanced Configuration](#Advanced_Configuration)
-* [Items for Future Work](#Items_for_Future_Work)
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
 
 
 <a name="Purpose"></a>Purpose

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ReservationSystem.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ReservationSystem.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ReservationSystem.md
index 7d1309f..cd4ae54 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ReservationSystem.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ReservationSystem.md
@@ -15,11 +15,7 @@
 Reservation System
 ==================
 
-
-* [Purpose](#Purpose)
-* [Overview](#Overview)
-* [Flow of a Reservation](#Flow_of_a_Reservation)
-* [Configuring the Reservation System](#Configuring_the_Reservation_System)
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
 
 Purpose
 -------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
index 49669a1..984959f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
@@ -15,15 +15,7 @@
 ResourceManager High Availability
 =================================
 
-* [Introduction](#Introduction)
-* [Architecture](#Architecture)
-    * [RM Failover](#RM_Failover)
-    * [Recovering prevous active-RM's state](#Recovering_prevous_active-RMs_state)
-* [Deployment](#Deployment)
-    * [Configurations](#Configurations)
-    * [Admin commands](#Admin_commands)
-    * [ResourceManager Web UI services](#ResourceManager_Web_UI_services)
-    * [Web Services](#Web_Services)
+<!-- MACRO{toc|fromDepth=0|toDepth=2} -->
 
 Introduction
 ------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
index 7881b5d..48dea5a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
@@ -15,32 +15,7 @@
 ResourceManager REST API's.
 ===========================
 
-* [Overview](#Overview)
-* [Enabling CORS support](#Enabling_CORS_support)
-* [Cluster Information API](#Cluster_Information_API)
-* [Cluster Metrics API](#Cluster_Metrics_API)
-* [Cluster Scheduler API](#Cluster_Scheduler_API)
-* [Cluster Applications API](#Cluster_Applications_API)
-* [Cluster Application Statistics API](#Cluster_Application_Statistics_API)
-* [Cluster Application API](#Cluster_Application_API)
-* [Cluster Application Attempts API](#Cluster_Application_Attempts_API)
-* [Cluster Nodes API](#Cluster_Nodes_API)
-* [Cluster Node API](#Cluster_Node_API)
-* [Cluster Writeable APIs](#Cluster_Writeable_APIs)
-* [Cluster New Application API](#Cluster_New_Application_API)
-* [Cluster Applications API(Submit Application)](#Cluster_Applications_APISubmit_Application)
-* [Cluster Application State API](#Cluster_Application_State_API)
-* [Cluster Application Queue API](#Cluster_Application_Queue_API)
-* [Cluster Application Priority API](#Cluster_Application_Priority_API)
-* [Cluster Delegation Tokens API](#Cluster_Delegation_Tokens_API)
-* [Cluster Reservation API List](#Cluster_Reservation_API_List)
-* [Cluster Reservation API Create](#Cluster_Reservation_API_Create)
-* [Cluster Reservation API Submit](#Cluster_Reservation_API_Submit)
-* [Cluster Reservation API Update](#Cluster_Reservation_API_Update)
-* [Cluster Reservation API Delete](#Cluster_Reservation_API_Delete)
-* [Cluster Application Timeouts API](#Cluster_Application_Timeouts_API)
-* [Cluster Application Timeout API](#Cluster_Application_Timeout_API)
-* [Cluster Application Timeout Update API](#Cluster_Application_Timeout_Update_API)
+<!-- MACRO{toc|fromDepth=0|toDepth=1} -->
 
 Overview
 --------
@@ -4280,4 +4255,4 @@ Response Body:
     <expiryTime>2016-11-27T09:36:16.678+05:30</expiryTime>
     <remainingTimeInSeconds>90</remainingTimeInSeconds>
 </timeout>
-```
\ No newline at end of file
+```

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md
index ee222c7..e18945d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md
@@ -15,18 +15,7 @@
 ResourceManger Restart
 ======================
 
-* [Overview](#Overview)
-* [Feature](#Feature)
-* [Configurations](#Configurations)
-    * [Enable RM Restart](#Enable_RM_Restart)
-    * [Configure the state-store for persisting the RM state](#Configure_the_state-store_for_persisting_the_RM_state)
-    * [How to choose the state-store implementation](#How_to_choose_the_state-store_implementation)
-    * [Configurations for Hadoop FileSystem based state-store implementation](#Configurations_for_Hadoop_FileSystem_based_state-store_implementation)
-    * [Configurations for ZooKeeper based state-store implementation](#Configurations_for_ZooKeeper_based_state-store_implementation)
-    * [Configurations for LevelDB based state-store implementation](#Configurations_for_LevelDB_based_state-store_implementation)
-    * [Configurations for work-preserving RM recovery](#Configurations_for_work-preserving_RM_recovery)
-* [Notes](#Notes)
-* [Sample Configurations](#Sample_Configurations)
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
 
 Overview
 --------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/SecureContainer.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/SecureContainer.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/SecureContainer.md
index f7706c7..a66c042 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/SecureContainer.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/SecureContainer.md
@@ -15,7 +15,7 @@
 YARN Secure Containers
 ======================
 
-* [Overview](#Overview)
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
 
 Overview
 --------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/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 ae9faae..ac9b2ec 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
@@ -15,21 +15,7 @@
 The YARN Timeline Server
 ========================
 
-* [Overview](#Overview)
-    * [Introduction](#Introduction)
-    * [Current Status](#Current_Status)
-    * [Timeline Structure](#Timeline_Structure)
-* [Deployment](#Deployment)
-    * [Configurations](#Configurations)
-    * [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)
-* [Timelnine Server Performance Test Tool](#TIMELINE_SERVER_PERFORMANCE_TEST_TOOL)
-    * [Highlights](#HIGHLIGHTS)
-    * [Usage](#USAGE)
-    * [Sample Runs](#SAMPLE_RUNS)
+<!-- MACRO{toc|fromDepth=1|toDepth=1} -->
 
 <a name="Overview"></a>Overview
 ---------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
index 61aa6ed..90c7a89 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
@@ -15,25 +15,9 @@
 The YARN Timeline Service v.2
 ========================
 
-* [Overview](#Overview)
-    * [Introduction](#Introduction)
-    * [Architecture](#Architecture)
-    * [Current Status](#Current_Status)
-* [Deployment](#Deployment)
-    * [Configurations](#Configurations)
-    * [Enabling Timeline Service v.2](#Enabling_Timeline_Service_v2)
-* [Publishing of application specific data](#Publishing_of_application_specific_data)
-* [Timeline Service v.2 REST API](#Timeline_Service_REST_API_v2)
-    * [Query Flows](#REST_API_LIST_FLOWS)
-    * [Query Flow Runs](#REST_API_LIST_FLOW_RUNS)
-    * [Query Flow Run](#REST_API_LIST_FLOW_RUN)
-    * [Query Apps for a Flow](#REST_API_LIST_FLOW_APPS)
-    * [Query Apps for a Flow Run](#REST_API_LIST_FLOWRUN_APPS)
-    * [Query App](#REST_API_LIST_APP)
-    * [Query Generic Entities](#REST_API_LIST_ENTITIES)
-    * [Query Generic Entity](#REST_API_LIST_ENTITY)
-
-#<a name="Overview"></a>Overview
+<!-- MACRO{toc|fromDepth=0|toDepth=2} -->
+
+## <a name="Overview"></a>Overview
 
 ### <a name="Introduction"></a>Introduction
 
@@ -128,9 +112,9 @@ analysis
 * Migration and compatibility with v.1
 
 
-#<a name="Deployment"></a>Deployment
+## <a name="Deployment"></a>Deployment
 
-###<a name="Configurations"></a>Configurations
+### <a name="Configurations"></a>Configurations
 
 New configuration parameters that are introduced with v.2 are marked bold.
 
@@ -269,7 +253,7 @@ To write MapReduce framework data to Timeline Service v.2, enable the following
 </property>
 ```
 
-###<a name="Publishing_of_application_specific_data"></a> Publishing application specific data
+### <a name="Publishing_of_application_specific_data"></a> Publishing application specific data
 
 This section is for YARN application developers that want to integrate with Timeline Service v.2.
 
@@ -402,7 +386,7 @@ You can provide the flow context via YARN application tags:
     appContext.setApplicationTags(tags);
 
 
-# <a name="Timeline_Service_REST_API_v2"></a>Timeline Service v.2 REST API
+## Timeline Service v.2 REST API
 
 Querying Timeline Service v.2 is currently only supported via REST API; there is no API client
 implemented in the YARN libraries.
@@ -430,14 +414,14 @@ Returns a JSON object describing the service instance and version information.
 
 The following shows the supported queries on the REST API.
 
-## <a name="REST_API_LIST_FLOWS"></a>Query Flows
+### <a name="REST_API_LIST_FLOWS"></a>Query Flows
 
 With Query Flows API, you can retrieve a list of active flows that had runs most recently.
 If the REST endpoint without the cluster name is used, the cluster specified by the configuration
 `yarn.resourcemanager.cluster-id` in `yarn-site.xml` is taken. If none of the flows match the
 predicates, an empty list will be returned.
 
-### HTTP request:
+#### HTTP request:
 
     GET /ws/v2/timeline/clusters/{cluster name}/flows/
 
@@ -445,7 +429,7 @@ predicates, an empty list will be returned.
 
     GET /ws/v2/timeline/flows/
 
-### Query Parameters Supported:
+#### Query Parameters Supported:
 
 1. `limit` - If specified, defines the number of flows to return. The maximum possible value for limit
   is maximum value of Long. If it is not specified or has a value less than 0, then limit will be
@@ -462,7 +446,7 @@ predicates, an empty list will be returned.
   "daterange=20150711-" returns flows active on and after 20150711.<br/>
   "daterange=-20150711" returns flows active on and before 20150711.<br/>
 
-### Example JSON Response:
+#### Example JSON Response:
 
     [
       {
@@ -515,14 +499,14 @@ predicates, an empty list will be returned.
       }
     ]
 
-### Response Codes
+#### Response Codes
 
 1. If successful, a HTTP 200 (OK) response is returned.
 1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned.
 1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned.
 
 
-## <a name="REST_API_LIST_FLOW_RUNS"></a>Query Flow Runs
+### <a name="REST_API_LIST_FLOW_RUNS"></a>Query Flow Runs
 
 With Query Flow Runs API, you can drill further down to get the runs (specific instances) of a
 given flow. This returns the most recent runs that belong to the given flow. If the REST
@@ -530,7 +514,7 @@ endpoint without the cluster name is used, the cluster specified by the configur
 `yarn.resourcemanager.cluster-id` in `yarn-site.xml` is taken. If none of the flow runs match the
 predicates, an empty list will be returned.
 
-### HTTP request:
+#### HTTP request:
 
     GET /ws/v2/timeline/clusters/{cluster name}/users/{user name}/flows/{flow name}/runs/
 
@@ -538,7 +522,7 @@ predicates, an empty list will be returned.
 
     GET /ws/v2/timeline/users/{user name}/flows/{flow name}/runs/
 
-### Query Parameters Supported:
+#### Query Parameters Supported:
 
 1. `limit` - If specified, defines the number of flows to return. The maximum possible value for limit
   is maximum value of Long. If it is not specified or has a value less than 0, then limit will be
@@ -558,7 +542,7 @@ predicates, an empty list will be returned.
   Other fields will lead to HTTP 400 (Bad Request) response. If not specified, in response, id, type, createdtime and info fields
   will be returned.
 
-### Example JSON Response:
+#### Example JSON Response:
 
     [
       {
@@ -595,21 +579,21 @@ predicates, an empty list will be returned.
       }
     ]
 
-### Response Codes
+#### Response Codes
 
 1. If successful, a HTTP 200 (OK) response is returned.
 1. If any problem occurs in parsing request or if an invalid field is specified in fields query param, HTTP 400 (Bad Request) is returned.
 1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned.
 
 
-## <a name="REST_API_LIST_FLOW_RUN"></a>Query Flow Run
+### <a name="REST_API_LIST_FLOW_RUN"></a>Query Flow Run
 
 With this API, you can query a specific flow run identified by cluster, user, flow name and run id.
 If the REST endpoint without the cluster name is used, the cluster specified by the configuration
 `yarn.resourcemanager.cluster-id` in `yarn-site.xml` is taken. Metrics are returned by default
 while querying individual flow runs.
 
-### HTTP request:
+#### HTTP request:
 
     GET /ws/v2/timeline/clusters/{cluster name}/users/{user name}/flows/{flow name}/runs/{run id}
 
@@ -617,7 +601,7 @@ while querying individual flow runs.
 
     GET /ws/v2/timeline/users/{user name}/flows/{flow name}/runs/{run id}
 
-### Query Parameters Supported:
+#### Query Parameters Supported:
 
 1. `metricstoretrieve` - If specified, defines which metrics to retrieve or which ones not to retrieve and send back in response.
   metricstoretrieve can be an expression of the form :<br/>
@@ -628,7 +612,7 @@ while querying individual flow runs.
   This specifies a comma separated list of metric id prefixes. Only metrics not matching any of the prefixes will be retrieved.<br/>
   Please note that URL unsafe characters such as spaces will have to be suitably encoded.
 
-### Example JSON Response:
+#### Example JSON Response:
 
     {
       "metrics": [
@@ -664,14 +648,14 @@ while querying individual flow runs.
       "relatesto": {}
     }
 
-### Response Codes
+#### Response Codes
 
 1. If successful, a HTTP 200(OK) response is returned.
 1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned.
 1. If flow run for the given flow run id cannot be found, HTTP 404 (Not Found) is returned.
 1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned.
 
-## <a name="REST_API_LIST_FLOW_APPS"></a>Query Apps for a flow
+### <a name="REST_API_LIST_FLOW_APPS"></a>Query Apps for a flow
 
 With this API, you can query all the YARN applications that are part of a specific flow. If the
 REST endpoint without the cluster name is used, the cluster specified by the configuration
@@ -679,7 +663,7 @@ REST endpoint without the cluster name is used, the cluster specified by the con
 applications are more than the limit, the most recent apps up to the limit will be returned. If
 none of the apps match the predicates, an empty list will be returned.
 
-### HTTP request:
+#### HTTP request:
 
     GET /ws/v2/timeline/clusters/{cluster name}/users/{user name}/flows/{flow name}/apps
 
@@ -687,7 +671,7 @@ none of the apps match the predicates, an empty list will be returned.
 
     GET /ws/v2/timeline/users/{user name}/flows/{flow name}/apps
 
-### Query Parameters Supported:
+#### Query Parameters Supported:
 
 1. `limit` - If specified, defines the number of applications to return. The maximum possible value for limit
   is maximum value of Long. If it is not specified or has a value less than 0, then limit will be
@@ -761,7 +745,7 @@ none of the apps match the predicates, an empty list will be returned.
   Integer. If it is not specified or has a value less than 1, and metrics have to be retrieved, then metricslimit will be
   considered as 1 i.e. latest single value of metric(s) will be returned.
 
-### Example JSON Response:
+#### Example JSON Response:
 
     [
       {
@@ -792,13 +776,13 @@ none of the apps match the predicates, an empty list will be returned.
       }
     ]
 
-### Response Codes
+#### Response Codes
 
 1. If successful, a HTTP 200 (OK) response is returned.
 1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned.
 1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned.
 
-## <a name="REST_API_LIST_FLOWRUN_APPS"></a>Query Apps for a flow run
+### <a name="REST_API_LIST_FLOWRUN_APPS"></a>Query Apps for a flow run
 
 With this API, you can query all the YARN applications that are part of a specific flow run. If the
 REST endpoint without the cluster name is used, the cluster specified by the configuration
@@ -806,7 +790,7 @@ REST endpoint without the cluster name is used, the cluster specified by the con
 are more than the limit, the most recent apps up to the limit will be returned. If none of the apps
 match the predicates, an empty list will be returned.
 
-### HTTP request:
+#### HTTP request:
 
     GET /ws/v2/timeline/clusters/{cluster name}/users/{user name}/flows/{flow name}/runs/{run id}/apps
 
@@ -814,7 +798,7 @@ match the predicates, an empty list will be returned.
 
     GET /ws/v2/timeline/users/{user name}/flows/{flow name}/runs/{run id}/apps/
 
-### Query Parameters Supported:
+#### Query Parameters Supported:
 
 1. `limit` - If specified, defines the number of applications to return. The maximum possible value for limit
   is maximum value of Long. If it is not specified or has a value less than 0, then limit will be
@@ -888,7 +872,7 @@ match the predicates, an empty list will be returned.
   Integer. If it is not specified or has a value less than 1, and metrics have to be retrieved, then metricslimit will be
   considered as 1 i.e. latest single value of metric(s) will be returned.
 
-### Example JSON Response:
+#### Example JSON Response:
 
     [
       {
@@ -906,14 +890,14 @@ match the predicates, an empty list will be returned.
       }
     ]
 
-### Response Codes
+#### Response Codes
 
 1. If successful, a HTTP 200 (OK) response is returned.
 1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned.
 1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned.
 
 
-## <a name="REST_API_LIST_APP"></a>Query app
+### <a name="REST_API_LIST_APP"></a>Query app
 
 With this API, you can query a single YARN application identified by the cluster and the
 application ID. If the REST endpoint without the cluster name is used, the cluster specified by the
@@ -922,7 +906,7 @@ information i.e. user, flow name and run id are not mandatory but if specified i
 preclude the need for an additional operation to fetch flow context information based on cluster
 and app id.
 
-### HTTP request:
+#### HTTP request:
 
     GET /ws/v2/timeline/clusters/{cluster name}/apps/{app id}
 
@@ -930,7 +914,7 @@ and app id.
 
     GET /ws/v2/timeline/apps/{app id}
 
-### Query Parameters Supported:
+#### Query Parameters Supported:
 
 1. `userid` -  If specified, only applications belonging to this user will be returned. This query param must be specified along with flowname and flowrunid query params, otherwise it will be ignored.
   If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query.
@@ -964,7 +948,7 @@ and app id.
   Integer. If it is not specified or has a value less than 1, and metrics have to be retrieved, then metricslimit will be
   considered as 1 i.e. latest single value of metric(s) will be returned.
 
-### Example JSON Response:
+#### Example JSON Response:
 
     {
       "metrics": [],
@@ -980,14 +964,14 @@ and app id.
       "relatesto": {}
     }
 
-### Response Codes
+#### Response Codes
 
 1. If successful, a HTTP 200(OK) response is returned.
 1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned.
 1. If flow context information cannot be retrieved or application for the given app id cannot be found, HTTP 404 (Not Found) is returned.
 1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned.
 
-## <a name="REST_API_LIST_ENTITIES"></a>Query generic entities
+### <a name="REST_API_LIST_ENTITIES"></a>Query generic entities
 
 With this API, you can query generic entities identified by cluster ID, application ID and
 per-framework entity type. If the REST endpoint without the cluster name is used, the cluster
@@ -1001,7 +985,7 @@ For instance, we can query containers by specifying entity type as `YARN_CONTAIN
 attempts by specifying entity type as `YARN_APPLICATION_ATTEMPT`.
 If none of the entities match the predicates, an empty list will be returned.
 
-### HTTP request:
+#### HTTP request:
 
     GET /ws/v2/timeline/clusters/{cluster name}/apps/{app id}/entities/{entity type}
 
@@ -1009,7 +993,7 @@ If none of the entities match the predicates, an empty list will be returned.
 
     GET /ws/v2/timeline/apps/{app id}/entities/{entity type}
 
-### Query Parameters Supported:
+#### Query Parameters Supported:
 
 1. `userid` -  If specified, only entities belonging to this user will be returned. This query param must be specified along with flowname and flowrunid query params, otherwise it will be ignored.
   If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query.
@@ -1088,7 +1072,7 @@ If none of the entities match the predicates, an empty list will be returned.
   Integer. If it is not specified or has a value less than 1, and metrics have to be retrieved, then metricslimit will be
   considered as 1 i.e. latest single value of metric(s) will be returned.
 
-### Example JSON Response:
+#### Example JSON Response:
 
     [
       {
@@ -1119,14 +1103,14 @@ If none of the entities match the predicates, an empty list will be returned.
       }
     ]
 
-### Response Codes
+#### Response Codes
 
 1. If successful, a HTTP 200(OK) response is returned.
 1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned.
 1. If flow context information cannot be retrieved, HTTP 404 (Not Found) is returned.
 1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned.
 
-## <a name="REST_API_LIST_ENTITY"></a>Query generic entity
+### <a name="REST_API_LIST_ENTITY"></a>Query generic entity
 
 With this API, you can query a specific generic entity identified by cluster ID, application ID,
 per-framework entity type and entity ID. If the REST endpoint without the cluster name is used, the
@@ -1139,7 +1123,7 @@ a specific YARN container by specifying entity type as `YARN_CONTAINER` and givi
 container ID. Similarly, application attempt can be queried by specifying entity type as
 `YARN_APPLICATION_ATTEMPT` and entity ID being the application attempt ID.
 
-### HTTP request:
+#### HTTP request:
 
     GET /ws/v2/timeline/clusters/{cluster name}/apps/{app id}/entities/{entity type}/{entity id}
 
@@ -1147,7 +1131,7 @@ container ID. Similarly, application attempt can be queried by specifying entity
 
     GET /ws/v2/timeline/apps/{app id}/entities/{entity type}/{entity id}
 
-### Query Parameters Supported:
+#### Query Parameters Supported:
 
 1. `userid` -  If specified, entity must belong to this user. This query param must be specified along with flowname and flowrunid query params, otherwise it will be ignored.
   If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query.
@@ -1181,7 +1165,7 @@ container ID. Similarly, application attempt can be queried by specifying entity
   Integer. If it is not specified or has a value less than 1, and metrics have to be retrieved, then metricslimit will be
   considered as 1 i.e. latest single value of metric(s) will be returned.
 
-### Example JSON Response:
+#### Example JSON Response:
 
     {
       "metrics": [ ],
@@ -1197,7 +1181,7 @@ container ID. Similarly, application attempt can be queried by specifying entity
       "relatesto": { }
     }
 
-### Response Codes
+#### Response Codes
 
 1. If successful, a HTTP 200 (OK) response is returned.
 1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebApplicationProxy.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebApplicationProxy.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebApplicationProxy.md
index 5fa9a9d..c3677cf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebApplicationProxy.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebApplicationProxy.md
@@ -15,12 +15,7 @@
 Web Application Proxy
 =====================
 
-* [Overview](#Overview)
-    * [Introduction](#Introduction)
-    * [Current Status](#Current_Status)
-* [Deployment](#Deployment)
-    * [Configurations](#Configurations)
-    * [Running Web Application Proxy](#Running_Web_Proxy)
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
 
 
 Overview

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebServicesIntro.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebServicesIntro.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebServicesIntro.md
index 0a396d3..11452f6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebServicesIntro.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WebServicesIntro.md
@@ -15,18 +15,7 @@
 Hadoop YARN - Introduction to the web services REST API's
 ==========================================================
 
-* [Overview](#Overview)
-* [URI's](#URIs)
-* [HTTP Requests](#HTTP_Requests)
-    * [Summary of HTTP operations](#Summary_of_HTTP_operations)
-    * [Security](#Security)
-    * [Headers Supported](#Headers_Supported)
-* [HTTP Responses](#HTTP_Responses)
-    * [Compression](#Compression)
-    * [Response Formats](#Response_Formats)
-    * [Response Errors](#Response_Errors)
-    * [Response Examples](#Response_Examples)
-* [Sample Usage](#Sample_Usage)
+<!-- MACRO{toc|fromDepth=0|toDepth=2} -->
 
 Overview
 --------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md
index cdad3f9..07c3765 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/WritingYarnApplications.md
@@ -15,19 +15,7 @@
 Hadoop: Writing YARN Applications
 =================================
 
-* [Purpose](#Purpose)
-* [Concepts and Flow](#Concepts_and_Flow)
-* [Interfaces](#Interfaces)
-* [Writing a Simple Yarn Application](#Writing_a_Simple_Yarn_Application)
-    * [Writing a simple Client](#Writing_a_simple_Client)
-    * [Writing an ApplicationMaster (AM)](#Writing_an_ApplicationMaster_AM)
-* [FAQ](#FAQ)
-    * [How can I distribute my application's jars to all of the nodes in the YARN cluster that need it?](#How_can_I_distribute_my_applications_jars_to_all_of_the_nodes_in_the_YARN_cluster_that_need_it)
-    * [How do I get the ApplicationMaster's ApplicationAttemptId?](#How_do_I_get_the_ApplicationMasters_ApplicationAttemptId)
-    * [Why my container is killed by the NodeManager?](#Why_my_container_is_killed_by_the_NodeManager)
-    * [How do I include native libraries?](#How_do_I_include_native_libraries)
-* [Useful Links](#Useful_Links)
-* [Sample Code](#Sample_Code)
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
 
 Purpose
 -------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnApplicationSecurity.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnApplicationSecurity.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnApplicationSecurity.md
index 640139f..bab46b9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnApplicationSecurity.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnApplicationSecurity.md
@@ -14,6 +14,8 @@
 
 # YARN Application Security
 
+<!-- MACRO{toc|fromDepth=0|toDepth=1} -->
+
 Anyone writing a YARN application needs to understand the process, in order
 to write short-lived applications or long-lived services. They also need to
 start testing on secure clusters during early development stages, in order

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbc0c2bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
index 708a8b4..478377fa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
@@ -15,31 +15,7 @@
 YARN Commands
 =============
 
-* [Overview](#Overview)
-* [User Commands](#User_Commands)
-    * [application](#application)
-    * [applicationattempt](#applicationattempt)
-    * [classpath](#classpath)
-    * [container](#container)
-    * [jar](#jar)
-    * [logs](#logs)
-    * [node](#node)
-    * [queue](#queue)
-    * [version](#version)
-* [Administration Commands](#Administration_Commands)
-    * [daemonlog](#daemonlog)
-    * [nodemanager](#nodemanager)
-    * [proxyserver](#proxyserver)
-    * [resourcemanager](#resourcemanager)
-    * [rmadmin](#rmadmin)
-    * [scmadmin](#scmadmin)
-    * [sharedcachemanager](#sharedcachemanager)
-    * [timelineserver](#timelineserver)
-* [Files](#Files)
-    * [etc/hadoop/hadoop-env.sh](#etchadoophadoop-env.sh)
-    * [etc/hadoop/yarn-env.sh](#etchadoopyarn-env.sh)
-    * [etc/hadoop/hadoop-user-functions.sh](#etchadoophadoop-user-functions.sh)
-    * [~/.hadooprc](#a.hadooprc)
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
 
 Overview
 --------


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


[12/50] [abbrv] hadoop git commit: HADOOP-14072. AliyunOSS: Failed to read from stream when seek beyond the download size. Contributed by Genmao Yu

Posted by xg...@apache.org.
HADOOP-14072. AliyunOSS: Failed to read from stream when seek beyond the download size. Contributed by Genmao Yu


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

Branch: refs/heads/YARN-5734
Commit: cd3e59a3dcc69f68711777d448da5228a55846b3
Parents: 8acb376
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Feb 15 16:34:30 2017 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Wed Feb 15 16:34:30 2017 +0800

----------------------------------------------------------------------
 .../fs/aliyun/oss/AliyunOSSInputStream.java     |  4 ++-
 .../oss/contract/TestAliyunOSSContractSeek.java | 26 ++++++++++++++++++++
 2 files changed, 29 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd3e59a3/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
index a3af7ce..72ba619 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
@@ -224,8 +224,10 @@ public class AliyunOSSInputStream extends FSInputStream {
     if (position == pos) {
       return;
     } else if (pos > position && pos < position + partRemaining) {
-      AliyunOSSUtils.skipFully(wrappedStream, pos - position);
+      long len = pos - position;
+      AliyunOSSUtils.skipFully(wrappedStream, len);
       position = pos;
+      partRemaining -= len;
     } else {
       reopen(pos);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd3e59a3/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractSeek.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractSeek.java
index b247ab1..d9b3674 100644
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractSeek.java
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/contract/TestAliyunOSSContractSeek.java
@@ -19,8 +19,15 @@
 package org.apache.hadoop.fs.aliyun.oss.contract;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.Test;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 
 /**
  * Aliyun OSS contract seeking tests.
@@ -31,4 +38,23 @@ public class TestAliyunOSSContractSeek extends AbstractContractSeekTest {
   protected AbstractFSContract createContract(Configuration conf) {
     return new AliyunOSSContract(conf);
   }
+
+  @Test
+  public void testSeekBeyondDownloadSize() throws Throwable {
+    describe("seek and read beyond download size.");
+
+    Path byteFile = path("byte_file.txt");
+    // 'fs.oss.multipart.download.size' = 100 * 1024
+    byte[] block = dataset(100 * 1024 + 10, 0, 255);
+    FileSystem fs = getFileSystem();
+    createFile(fs, byteFile, true, block);
+
+    FSDataInputStream instream = getFileSystem().open(byteFile);
+    instream.seek(100 * 1024 - 1);
+    assertEquals(100 * 1024 - 1, instream.getPos());
+    assertEquals(144, instream.read());
+    instream.seek(100 * 1024 + 1);
+    assertEquals(100 * 1024 + 1, instream.getPos());
+    assertEquals(146, instream.read());
+  }
 }


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


[36/50] [abbrv] hadoop git commit: YARN-6188. Fix OOM issue with decommissioningNodesWatcher in the case of clusters with large number of nodes (Contributed by Ajay Jadhav via Daniel Templeton)

Posted by xg...@apache.org.
YARN-6188. Fix OOM issue with decommissioningNodesWatcher in the case of clusters with
large number of nodes (Contributed by Ajay Jadhav via Daniel Templeton)


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

Branch: refs/heads/YARN-5734
Commit: 9a928377868dfb2dc846c340501b3248eb6ad77f
Parents: 4c26c24
Author: Daniel Templeton <te...@apache.org>
Authored: Fri Feb 17 13:11:43 2017 -0800
Committer: Daniel Templeton <te...@apache.org>
Committed: Fri Feb 17 13:13:46 2017 -0800

----------------------------------------------------------------------
 .../yarn/server/resourcemanager/DecommissioningNodesWatcher.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a928377/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DecommissioningNodesWatcher.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/DecommissioningNodesWatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DecommissioningNodesWatcher.java
index 376b503..9631803 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DecommissioningNodesWatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DecommissioningNodesWatcher.java
@@ -385,9 +385,9 @@ public class DecommissioningNodesWatcher {
     if (!LOG.isDebugEnabled() || decomNodes.size() == 0) {
       return;
     }
-    StringBuilder sb = new StringBuilder();
     long now = mclock.getTime();
     for (DecommissioningNodeContext d : decomNodes.values()) {
+      StringBuilder sb = new StringBuilder();
       DecommissioningNodeStatus s = checkDecommissioningStatus(d.nodeId);
       sb.append(String.format(
           "%n  %-34s %4ds fresh:%3ds containers:%2d %14s",
@@ -413,8 +413,8 @@ public class DecommissioningNodesWatcher {
               (mclock.getTime() - rmApp.getStartTime()) / 1000));
         }
       }
+      LOG.debug("Decommissioning node: " + sb.toString());
     }
-    LOG.info("Decommissioning Nodes: " + sb.toString());
   }
 
   // Read possible new DECOMMISSIONING_TIMEOUT_KEY from yarn-site.xml.


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


[37/50] [abbrv] hadoop git commit: YARN-6125. The application attempt's diagnostic message should have a maximum size (Contributed by Andras Piros via Daniel Templeton)

Posted by xg...@apache.org.
YARN-6125. The application attempt's diagnostic message should have a maximum size
(Contributed by Andras Piros via Daniel Templeton)


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

Branch: refs/heads/YARN-5734
Commit: c7a36e613053ec8b111146004b887c2f13535469
Parents: 9a92837
Author: Daniel Templeton <te...@apache.org>
Authored: Fri Feb 17 13:40:58 2017 -0800
Committer: Daniel Templeton <te...@apache.org>
Committed: Fri Feb 17 13:40:58 2017 -0800

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   5 +
 .../src/main/resources/yarn-default.xml         |  15 ++
 .../hadoop-yarn-server-resourcemanager/pom.xml  |  16 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         | 172 ++++++++++++++++++-
 .../rmapp/attempt/TestBoundedAppender.java      | 116 +++++++++++++
 .../TestRMAppAttemptImplDiagnostics.java        | 111 ++++++++++++
 6 files changed, 422 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7a36e61/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 136227a..094a424 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
@@ -2619,6 +2619,11 @@ public class YarnConfiguration extends Configuration {
 
   public static final int DEFAULT_CLUSTER_LEVEL_APPLICATION_PRIORITY = 0;
 
+  public static final String APP_ATTEMPT_DIAGNOSTICS_LIMIT_KC =
+      YARN_PREFIX + "app.attempt.diagnostics.limit.kc";
+
+  public static final int DEFAULT_APP_ATTEMPT_DIAGNOSTICS_LIMIT_KC = 64;
+
   @Private
   public static boolean isDistributedNodeLabelConfiguration(Configuration conf) {
     return DISTRIBUTED_NODELABEL_CONFIGURATION_TYPE.equals(conf.get(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7a36e61/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 4ca46f9..53beb5e 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
@@ -3022,4 +3022,19 @@
     <value>3000</value>
   </property>
 
+  <property>
+    <description>
+      Defines the limit of the diagnostics message of an application
+      attempt, in kilo characters (character count * 1024).
+      When using ZooKeeper to store application state behavior, it's
+      important to limit the size of the diagnostic messages to
+      prevent YARN from overwhelming ZooKeeper. In cases where
+      yarn.resourcemanager.state-store.max-completed-applications is set to
+      a large number, it may be desirable to reduce the value of this property
+      to limit the total data stored.
+    </description>
+    <name>yarn.app.attempt.diagnostics.limit.kc</name>
+    <value>64</value>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7a36e61/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
index 6985d65..0a85d0c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
@@ -52,8 +52,17 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-annotations</artifactId>
     </dependency>
+    <!--
+    junit must be before mockito-all on the classpath.  mockito-all bundles its
+    own copy of the hamcrest classes, but they don't match our junit version.
+    -->
     <dependency>
-      <groupId>org.mockito</groupId>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+    <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
       <scope>test</scope>
     </dependency>
@@ -73,11 +82,6 @@
       <artifactId>protobuf-java</artifactId>
     </dependency>
     <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>commons-io</groupId>
       <artifactId>commons-io</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7a36e61/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
index 1788722..25138c5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
@@ -38,6 +38,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
 import javax.crypto.SecretKey;
 
+import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -119,6 +120,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       "%s State change from %s to %s on event = %s";
   private static final String RECOVERY_MESSAGE =
       "Recovering attempt: %s with final state = %s";
+  private static final String DIAGNOSTIC_LIMIT_CONFIG_ERROR_MESSAGE =
+      "The value of %s should be a positive integer: %s";
 
   private static final Log LOG = LogFactory.getLog(RMAppAttemptImpl.class);
 
@@ -127,6 +130,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
 
   public final static Priority AM_CONTAINER_PRIORITY = recordFactory
       .newRecordInstance(Priority.class);
+
   static {
     AM_CONTAINER_PRIORITY.setPriority(0);
   }
@@ -171,7 +175,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
   // Set to null initially. Will eventually get set
   // if an RMAppAttemptUnregistrationEvent occurs
   private FinalApplicationStatus finalStatus = null;
-  private final StringBuilder diagnostics = new StringBuilder();
+  private final BoundedAppender diagnostics;
   private int amContainerExitStatus = ContainerExitStatus.INVALID;
 
   private Configuration conf;
@@ -518,6 +522,45 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
 
     this.amReq = amReq;
     this.blacklistedNodesForAM = amBlacklistManager;
+
+    final int diagnosticsLimitKC = getDiagnosticsLimitKCOrThrow(conf);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(YarnConfiguration.APP_ATTEMPT_DIAGNOSTICS_LIMIT_KC + " : " +
+              diagnosticsLimitKC);
+    }
+
+    this.diagnostics = new BoundedAppender(diagnosticsLimitKC * 1024);
+  }
+
+  private int getDiagnosticsLimitKCOrThrow(final Configuration configuration) {
+    try {
+      final int diagnosticsLimitKC = configuration.getInt(
+          YarnConfiguration.APP_ATTEMPT_DIAGNOSTICS_LIMIT_KC,
+          YarnConfiguration.DEFAULT_APP_ATTEMPT_DIAGNOSTICS_LIMIT_KC);
+
+      if (diagnosticsLimitKC <= 0) {
+        final String message =
+            String.format(DIAGNOSTIC_LIMIT_CONFIG_ERROR_MESSAGE,
+                YarnConfiguration.APP_ATTEMPT_DIAGNOSTICS_LIMIT_KC,
+                diagnosticsLimitKC);
+        LOG.error(message);
+
+        throw new YarnRuntimeException(message);
+      }
+
+      return diagnosticsLimitKC;
+    } catch (final NumberFormatException ignored) {
+      final String diagnosticsLimitKCString = configuration
+          .get(YarnConfiguration.APP_ATTEMPT_DIAGNOSTICS_LIMIT_KC);
+      final String message =
+          String.format(DIAGNOSTIC_LIMIT_CONFIG_ERROR_MESSAGE,
+              YarnConfiguration.APP_ATTEMPT_DIAGNOSTICS_LIMIT_KC,
+              diagnosticsLimitKCString);
+      LOG.error(message);
+
+      throw new YarnRuntimeException(message);
+    }
   }
 
   @Override
@@ -738,6 +781,11 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     }
   }
 
+  @VisibleForTesting
+  void appendDiagnostics(final CharSequence message) {
+    this.diagnostics.append(message);
+  }
+
   public int getAMContainerExitStatus() {
     this.readLock.lock();
     try {
@@ -926,8 +974,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
           attemptState.getState()));
     }
 
-    diagnostics.append("Attempt recovered after RM restart");
-    diagnostics.append(attemptState.getDiagnostics());
+    this.diagnostics.append("Attempt recovered after RM restart");
+    this.diagnostics.append(attemptState.getDiagnostics());
     this.amContainerExitStatus = attemptState.getAMContainerExitStatus();
     if (amContainerExitStatus == ContainerExitStatus.PREEMPTED) {
       this.attemptMetrics.setIsPreempted();
@@ -942,7 +990,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     this.startTime = attemptState.getStartTime();
     this.finishTime = attemptState.getFinishTime();
     this.attemptMetrics.updateAggregateAppResourceUsage(
-        attemptState.getMemorySeconds(),attemptState.getVcoreSeconds());
+        attemptState.getMemorySeconds(), attemptState.getVcoreSeconds());
     this.attemptMetrics.updateAggregatePreemptedAppResourceUsage(
         attemptState.getPreemptedMemorySeconds(),
         attemptState.getPreemptedVcoreSeconds());
@@ -1655,8 +1703,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
   private void setAMContainerCrashedDiagnosticsAndExitStatus(
       RMAppAttemptContainerFinishedEvent finishEvent) {
     ContainerStatus status = finishEvent.getContainerStatus();
-    String diagnostics = getAMContainerCrashedDiagnostics(finishEvent);
-    this.diagnostics.append(diagnostics);
+    this.diagnostics.append(getAMContainerCrashedDiagnostics(finishEvent));
     this.amContainerExitStatus = status.getExitStatus();
   }
 
@@ -1825,7 +1872,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     progress = 1.0f;
     RMAppAttemptUnregistrationEvent unregisterEvent =
         (RMAppAttemptUnregistrationEvent) event;
-    diagnostics.append(unregisterEvent.getDiagnosticMsg());
+    this.diagnostics.append(unregisterEvent.getDiagnosticMsg());
     originalTrackingUrl = sanitizeTrackingUrl(unregisterEvent.getFinalTrackingUrl());
     finalStatus = unregisterEvent.getFinalApplicationStatus();
   }
@@ -2232,4 +2279,115 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     }
     return Collections.EMPTY_SET;
   }
+
+  /**
+   * A {@link CharSequence} appender that considers its {@link #limit} as upper
+   * bound.
+   * <p>
+   * When {@link #limit} would be reached on append, past messages will be
+   * truncated from head, and a header telling the user about truncation will be
+   * prepended, with ellipses in between header and messages.
+   * <p>
+   * Note that header and ellipses are not counted against {@link #limit}.
+   * <p>
+   * An example:
+   *
+   * <pre>
+   * {@code
+   *   // At the beginning it's an empty string
+   *   final Appendable shortAppender = new BoundedAppender(80);
+   *   // The whole message fits into limit
+   *   shortAppender.append(
+   *       "message1 this is a very long message but fitting into limit\n");
+   *   // The first message is truncated, the second not
+   *   shortAppender.append("message2 this is shorter than the previous one\n");
+   *   // The first message is deleted, the second truncated, the third
+   *   // preserved
+   *   shortAppender.append("message3 this is even shorter message, maybe.\n");
+   *   // The first two are deleted, the third one truncated, the last preserved
+   *   shortAppender.append("message4 the shortest one, yet the greatest :)");
+   *   // Current contents are like this:
+   *   // Diagnostic messages truncated, showing last 80 chars out of 199:
+   *   // ...s is even shorter message, maybe.
+   *   // message4 the shortest one, yet the greatest :)
+   * }
+   * </pre>
+   * <p>
+   * Note that <tt>null</tt> values are {@link #append(CharSequence) append}ed
+   * just like in {@link StringBuilder#append(CharSequence) original
+   * implementation}.
+   * <p>
+   * Note that this class is not thread safe.
+   */
+  @VisibleForTesting
+  static class BoundedAppender {
+    @VisibleForTesting
+    static final String TRUNCATED_MESSAGES_TEMPLATE =
+        "Diagnostic messages truncated, showing last "
+            + "%d chars out of %d:%n...%s";
+
+    private final int limit;
+    private final StringBuilder messages = new StringBuilder();
+    private int totalCharacterCount = 0;
+
+    BoundedAppender(final int limit) {
+      Preconditions.checkArgument(limit > 0, "limit should be positive");
+
+      this.limit = limit;
+    }
+
+    /**
+     * Append a {@link CharSequence} considering {@link #limit}, truncating
+     * from the head of {@code csq} or {@link #messages} when necessary.
+     *
+     * @param csq the {@link CharSequence} to append
+     * @return this
+     */
+    BoundedAppender append(final CharSequence csq) {
+      appendAndCount(csq);
+      checkAndCut();
+
+      return this;
+    }
+
+    private void appendAndCount(final CharSequence csq) {
+      final int before = messages.length();
+      messages.append(csq);
+      final int after = messages.length();
+      totalCharacterCount += after - before;
+    }
+
+    private void checkAndCut() {
+      if (messages.length() > limit) {
+        final int newStart = messages.length() - limit;
+        messages.delete(0, newStart);
+      }
+    }
+
+    /**
+     * Get current length of messages considering truncates
+     * without header and ellipses.
+     *
+     * @return current length
+     */
+    int length() {
+      return messages.length();
+    }
+
+    /**
+     * Get a string representation of the actual contents, displaying also a
+     * header and ellipses when there was a truncate.
+     *
+     * @return String representation of the {@link #messages}
+     */
+    @Override
+    public String toString() {
+      if (messages.length() < totalCharacterCount) {
+        return String.format(TRUNCATED_MESSAGES_TEMPLATE, messages.length(),
+            totalCharacterCount, messages.toString());
+      }
+
+      return messages.toString();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7a36e61/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestBoundedAppender.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestBoundedAppender.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestBoundedAppender.java
new file mode 100644
index 0000000..9cb1e04
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestBoundedAppender.java
@@ -0,0 +1,116 @@
+/**
+ * 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.rmapp.attempt;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import static org.junit.Assert.assertEquals;
+import static org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.BoundedAppender;
+
+/**
+ * Test class for {@link BoundedAppender}.
+ */
+public class TestBoundedAppender {
+  @Rule
+  public ExpectedException expected = ExpectedException.none();
+
+  @Test
+  public void initWithZeroLimitThrowsException() {
+    expected.expect(IllegalArgumentException.class);
+    expected.expectMessage("limit should be positive");
+
+    new BoundedAppender(0);
+  }
+
+  @Test
+  public void nullAppendedNullStringRead() {
+    final BoundedAppender boundedAppender = new BoundedAppender(4);
+    boundedAppender.append(null);
+
+    assertEquals("null appended, \"null\" read", "null",
+        boundedAppender.toString());
+  }
+
+  @Test
+  public void appendBelowLimitOnceValueIsReadCorrectly() {
+    final BoundedAppender boundedAppender = new BoundedAppender(2);
+
+    boundedAppender.append("ab");
+
+    assertEquals("value appended is read correctly", "ab",
+        boundedAppender.toString());
+  }
+
+  @Test
+  public void appendValuesBelowLimitAreReadCorrectlyInFifoOrder() {
+    final BoundedAppender boundedAppender = new BoundedAppender(3);
+
+    boundedAppender.append("ab");
+    boundedAppender.append("cd");
+    boundedAppender.append("e");
+    boundedAppender.append("fg");
+
+    assertEquals("last values appended fitting limit are read correctly",
+        String.format(BoundedAppender.TRUNCATED_MESSAGES_TEMPLATE, 3, 7, "efg"),
+        boundedAppender.toString());
+  }
+
+  @Test
+  public void appendLastAboveLimitPreservesLastMessagePostfix() {
+    final BoundedAppender boundedAppender = new BoundedAppender(3);
+
+    boundedAppender.append("ab");
+    boundedAppender.append("cde");
+    boundedAppender.append("fghij");
+
+    assertEquals(
+        "last value appended above limit postfix is read correctly", String
+            .format(BoundedAppender.TRUNCATED_MESSAGES_TEMPLATE, 3, 10, "hij"),
+        boundedAppender.toString());
+  }
+
+  @Test
+  public void appendMiddleAboveLimitPreservesLastMessageAndMiddlePostfix() {
+    final BoundedAppender boundedAppender = new BoundedAppender(3);
+
+    boundedAppender.append("ab");
+    boundedAppender.append("cde");
+
+    assertEquals("last value appended above limit postfix is read correctly",
+        String.format(BoundedAppender.TRUNCATED_MESSAGES_TEMPLATE, 3, 5, "cde"),
+        boundedAppender.toString());
+
+    boundedAppender.append("fg");
+
+    assertEquals(
+        "middle value appended above limit postfix and last value are "
+            + "read correctly",
+        String.format(BoundedAppender.TRUNCATED_MESSAGES_TEMPLATE, 3, 7, "efg"),
+        boundedAppender.toString());
+
+    boundedAppender.append("hijkl");
+
+    assertEquals(
+        "last value appended above limit postfix is read correctly", String
+            .format(BoundedAppender.TRUNCATED_MESSAGES_TEMPLATE, 3, 12, "jkl"),
+        boundedAppender.toString());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7a36e61/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptImplDiagnostics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptImplDiagnostics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptImplDiagnostics.java
new file mode 100644
index 0000000..a160eb8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptImplDiagnostics.java
@@ -0,0 +1,111 @@
+/**
+ * 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.rmapp.attempt;
+
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Testing {@link RMAppAttemptImpl#diagnostics} scenarios.
+ */
+public class TestRMAppAttemptImplDiagnostics {
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  @Test
+  public void whenCreatedWithDefaultConfigurationSuccess() {
+    final Configuration configuration = new Configuration();
+    configuration.setInt(YarnConfiguration.APP_ATTEMPT_DIAGNOSTICS_LIMIT_KC,
+        YarnConfiguration.DEFAULT_APP_ATTEMPT_DIAGNOSTICS_LIMIT_KC);
+
+    createRMAppAttemptImpl(configuration);
+  }
+
+  @Test
+  public void whenCreatedWithWrongConfigurationError() {
+    final Configuration configuration = new Configuration();
+    configuration.setInt(YarnConfiguration.APP_ATTEMPT_DIAGNOSTICS_LIMIT_KC, 0);
+
+    expectedException.expect(YarnRuntimeException.class);
+
+    createRMAppAttemptImpl(configuration);
+  }
+
+  @Test
+  public void whenAppendedWithinLimitMessagesArePreserved() {
+    final Configuration configuration = new Configuration();
+    configuration.setInt(YarnConfiguration.APP_ATTEMPT_DIAGNOSTICS_LIMIT_KC, 1);
+
+    final RMAppAttemptImpl appAttempt = createRMAppAttemptImpl(configuration);
+
+    final String withinLimit = RandomStringUtils.random(1024);
+    appAttempt.appendDiagnostics(withinLimit);
+
+    assertEquals("messages within limit should be preserved", withinLimit,
+        appAttempt.getDiagnostics());
+  }
+
+  @Test
+  public void whenAppendedBeyondLimitMessagesAreTruncated() {
+    final Configuration configuration = new Configuration();
+    configuration.setInt(YarnConfiguration.APP_ATTEMPT_DIAGNOSTICS_LIMIT_KC, 1);
+
+    final RMAppAttemptImpl appAttempt = createRMAppAttemptImpl(configuration);
+
+    final String beyondLimit = RandomStringUtils.random(1025);
+    appAttempt.appendDiagnostics(beyondLimit);
+
+    final String truncated = String.format(
+        RMAppAttemptImpl.BoundedAppender.TRUNCATED_MESSAGES_TEMPLATE, 1024,
+        1025, beyondLimit.substring(1));
+
+    assertEquals("messages beyond limit should be truncated", truncated,
+        appAttempt.getDiagnostics());
+  }
+
+  private RMAppAttemptImpl createRMAppAttemptImpl(
+      final Configuration configuration) {
+    final ApplicationAttemptId mockApplicationAttemptId =
+        mock(ApplicationAttemptId.class);
+    final ApplicationId mockApplicationId = mock(ApplicationId.class);
+    when(mockApplicationAttemptId.getApplicationId())
+        .thenReturn(mockApplicationId);
+
+    final RMContext mockRMContext = mock(RMContext.class);
+    final Dispatcher mockDispatcher = mock(Dispatcher.class);
+    when(mockRMContext.getDispatcher()).thenReturn(mockDispatcher);
+
+    return new RMAppAttemptImpl(mockApplicationAttemptId, mockRMContext, null,
+        null, null, configuration, false, null);
+  }
+}
\ No newline at end of file


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


[44/50] [abbrv] hadoop git commit: HDFS-11404. Increase timeout on TestShortCircuitLocalRead.testDeprecatedGetBlockLocalPathInfoRpc. Contributed by Eric Badger

Posted by xg...@apache.org.
HDFS-11404. Increase timeout on TestShortCircuitLocalRead.testDeprecatedGetBlockLocalPathInfoRpc. Contributed by Eric Badger


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

Branch: refs/heads/YARN-5734
Commit: a07ddef10115fd0082832f1c338b2484507a8f49
Parents: 4804050
Author: Eric Payne <ep...@apache.org>
Authored: Tue Feb 21 12:04:25 2017 -0600
Committer: Eric Payne <ep...@apache.org>
Committed: Tue Feb 21 12:04:25 2017 -0600

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/shortcircuit/TestShortCircuitLocalRead.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a07ddef1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitLocalRead.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitLocalRead.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitLocalRead.java
index a7132b8..55e9795 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitLocalRead.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitLocalRead.java
@@ -353,7 +353,7 @@ public class TestShortCircuitLocalRead {
     });
   }
 
-  @Test(timeout=10000)
+  @Test(timeout=60000)
   public void testDeprecatedGetBlockLocalPathInfoRpc() throws IOException {
     final Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)


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


[27/50] [abbrv] hadoop git commit: YARN-4675. Reorganize TimelineClient and TimelineClientImpl into separate classes for ATSv1.x and ATSv2. Contributed by Naganarasimha G R.

Posted by xg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineConnector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineConnector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineConnector.java
new file mode 100644
index 0000000..b5b5f77
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineConnector.java
@@ -0,0 +1,440 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.client.api.impl;
+
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.net.ConnectException;
+import java.net.HttpURLConnection;
+import java.net.SocketTimeoutException;
+import java.net.URI;
+import java.net.URL;
+import java.net.URLConnection;
+import java.security.GeneralSecurityException;
+import java.security.PrivilegedExceptionAction;
+
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL.Token;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator;
+import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticator;
+import org.apache.hadoop.security.token.delegation.web.PseudoDelegationTokenAuthenticator;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientHandlerException;
+import com.sun.jersey.api.client.ClientRequest;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
+import com.sun.jersey.api.client.filter.ClientFilter;
+import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
+import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
+
+/**
+ * Utility Connector class which is used by timeline clients to securely get
+ * connected to the timeline server.
+ *
+ */
+public class TimelineConnector extends AbstractService {
+
+  private static final Joiner JOINER = Joiner.on("");
+  private static final Log LOG = LogFactory.getLog(TimelineConnector.class);
+  public final static int DEFAULT_SOCKET_TIMEOUT = 1 * 60 * 1000; // 1 minute
+
+  private SSLFactory sslFactory;
+  private Client client;
+  private ConnectionConfigurator connConfigurator;
+  private DelegationTokenAuthenticator authenticator;
+  private DelegationTokenAuthenticatedURL.Token token;
+  private UserGroupInformation authUgi;
+  private String doAsUser;
+  @VisibleForTesting
+  TimelineClientConnectionRetry connectionRetry;
+  private boolean requireConnectionRetry;
+
+  public TimelineConnector(boolean requireConnectionRetry,
+      UserGroupInformation authUgi, String doAsUser,
+      DelegationTokenAuthenticatedURL.Token token) {
+    super("TimelineConnector");
+    this.requireConnectionRetry = requireConnectionRetry;
+    this.authUgi = authUgi;
+    this.doAsUser = doAsUser;
+    this.token = token;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    ClientConfig cc = new DefaultClientConfig();
+    cc.getClasses().add(YarnJacksonJaxbJsonProvider.class);
+
+    sslFactory = getSSLFactory(conf);
+    connConfigurator = getConnConfigurator(sslFactory);
+
+    if (UserGroupInformation.isSecurityEnabled()) {
+      authenticator = new KerberosDelegationTokenAuthenticator();
+    } else {
+      authenticator = new PseudoDelegationTokenAuthenticator();
+    }
+    authenticator.setConnectionConfigurator(connConfigurator);
+
+    connectionRetry = new TimelineClientConnectionRetry(conf);
+    client =
+        new Client(
+            new URLConnectionClientHandler(new TimelineURLConnectionFactory(
+                authUgi, authenticator, connConfigurator, token, doAsUser)),
+            cc);
+    if (requireConnectionRetry) {
+      TimelineJerseyRetryFilter retryFilter =
+          new TimelineJerseyRetryFilter(connectionRetry);
+      client.addFilter(retryFilter);
+    }
+  }
+
+  private static final ConnectionConfigurator DEFAULT_TIMEOUT_CONN_CONFIGURATOR
+    = new ConnectionConfigurator() {
+        @Override
+        public HttpURLConnection configure(HttpURLConnection conn)
+            throws IOException {
+          setTimeouts(conn, DEFAULT_SOCKET_TIMEOUT);
+          return conn;
+        }
+      };
+
+  private ConnectionConfigurator getConnConfigurator(SSLFactory sslFactoryObj) {
+    try {
+      return initSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, sslFactoryObj);
+    } catch (Exception e) {
+      LOG.debug("Cannot load customized ssl related configuration. "
+          + "Fallback to system-generic settings.", e);
+      return DEFAULT_TIMEOUT_CONN_CONFIGURATOR;
+    }
+  }
+
+  private static ConnectionConfigurator initSslConnConfigurator(
+      final int timeout, SSLFactory sslFactory)
+      throws IOException, GeneralSecurityException {
+    final SSLSocketFactory sf;
+    final HostnameVerifier hv;
+
+    sf = sslFactory.createSSLSocketFactory();
+    hv = sslFactory.getHostnameVerifier();
+
+    return new ConnectionConfigurator() {
+      @Override
+      public HttpURLConnection configure(HttpURLConnection conn)
+          throws IOException {
+        if (conn instanceof HttpsURLConnection) {
+          HttpsURLConnection c = (HttpsURLConnection) conn;
+          c.setSSLSocketFactory(sf);
+          c.setHostnameVerifier(hv);
+        }
+        setTimeouts(conn, timeout);
+        return conn;
+      }
+    };
+  }
+
+  protected SSLFactory getSSLFactory(Configuration conf)
+      throws GeneralSecurityException, IOException {
+    SSLFactory newSSLFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
+    newSSLFactory.init();
+    return newSSLFactory;
+  }
+
+  private static void setTimeouts(URLConnection connection, int socketTimeout) {
+    connection.setConnectTimeout(socketTimeout);
+    connection.setReadTimeout(socketTimeout);
+  }
+
+  public static URI constructResURI(Configuration conf, String address,
+      String uri) {
+    return URI.create(
+        JOINER.join(YarnConfiguration.useHttps(conf) ? "https://" : "http://",
+            address, uri));
+  }
+
+  DelegationTokenAuthenticatedURL getDelegationTokenAuthenticatedURL() {
+    return new DelegationTokenAuthenticatedURL(authenticator, connConfigurator);
+  }
+
+  protected void serviceStop() {
+    if (this.sslFactory != null) {
+      this.sslFactory.destroy();
+    }
+  }
+
+  public Client getClient() {
+    return client;
+  }
+
+  public Object operateDelegationToken(
+      final PrivilegedExceptionAction<?> action)
+      throws IOException, YarnException {
+    // Set up the retry operation
+    TimelineClientRetryOp tokenRetryOp =
+        createRetryOpForOperateDelegationToken(action);
+
+    return connectionRetry.retryOn(tokenRetryOp);
+  }
+
+  @Private
+  @VisibleForTesting
+  TimelineClientRetryOp createRetryOpForOperateDelegationToken(
+      final PrivilegedExceptionAction<?> action) throws IOException {
+    return new TimelineClientRetryOpForOperateDelegationToken(this.authUgi,
+        action);
+  }
+
+  /**
+   * Abstract class for an operation that should be retried by timeline client.
+   */
+  @Private
+  @VisibleForTesting
+  public static abstract class TimelineClientRetryOp {
+    // The operation that should be retried
+    public abstract Object run() throws IOException;
+
+    // The method to indicate if we should retry given the incoming exception
+    public abstract boolean shouldRetryOn(Exception e);
+  }
+
+  private static class TimelineURLConnectionFactory
+      implements HttpURLConnectionFactory {
+    private DelegationTokenAuthenticator authenticator;
+    private UserGroupInformation authUgi;
+    private ConnectionConfigurator connConfigurator;
+    private Token token;
+    private String doAsUser;
+
+    public TimelineURLConnectionFactory(UserGroupInformation authUgi,
+        DelegationTokenAuthenticator authenticator,
+        ConnectionConfigurator connConfigurator,
+        DelegationTokenAuthenticatedURL.Token token, String doAsUser) {
+      this.authUgi = authUgi;
+      this.authenticator = authenticator;
+      this.connConfigurator = connConfigurator;
+      this.token = token;
+      this.doAsUser = doAsUser;
+    }
+
+    @Override
+    public HttpURLConnection getHttpURLConnection(final URL url)
+        throws IOException {
+      authUgi.checkTGTAndReloginFromKeytab();
+      try {
+        return new DelegationTokenAuthenticatedURL(authenticator,
+            connConfigurator).openConnection(url, token, doAsUser);
+      } catch (UndeclaredThrowableException e) {
+        throw new IOException(e.getCause());
+      } catch (AuthenticationException ae) {
+        throw new IOException(ae);
+      }
+    }
+
+  }
+
+  // Class to handle retry
+  // Outside this class, only visible to tests
+  @Private
+  @VisibleForTesting
+  static class TimelineClientConnectionRetry {
+
+    // maxRetries < 0 means keep trying
+    @Private
+    @VisibleForTesting
+    public int maxRetries;
+
+    @Private
+    @VisibleForTesting
+    public long retryInterval;
+
+    // Indicates if retries happened last time. Only tests should read it.
+    // In unit tests, retryOn() calls should _not_ be concurrent.
+    private boolean retried = false;
+
+    @Private
+    @VisibleForTesting
+    boolean getRetired() {
+      return retried;
+    }
+
+    // Constructor with default retry settings
+    public TimelineClientConnectionRetry(Configuration conf) {
+      Preconditions.checkArgument(
+          conf.getInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
+              YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES)
+              >= -1,
+          "%s property value should be greater than or equal to -1",
+          YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES);
+      Preconditions.checkArgument(
+          conf.getLong(
+              YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
+              YarnConfiguration.
+                DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS) > 0,
+          "%s property value should be greater than zero",
+          YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS);
+      maxRetries =
+          conf.getInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
+              YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES);
+      retryInterval = conf.getLong(
+          YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
+          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS);
+    }
+
+    public Object retryOn(TimelineClientRetryOp op)
+        throws RuntimeException, IOException {
+      int leftRetries = maxRetries;
+      retried = false;
+
+      // keep trying
+      while (true) {
+        try {
+          // try perform the op, if fail, keep retrying
+          return op.run();
+        } catch (IOException | RuntimeException e) {
+          // break if there's no retries left
+          if (leftRetries == 0) {
+            break;
+          }
+          if (op.shouldRetryOn(e)) {
+            logException(e, leftRetries);
+          } else {
+            throw e;
+          }
+        }
+        if (leftRetries > 0) {
+          leftRetries--;
+        }
+        retried = true;
+        try {
+          // sleep for the given time interval
+          Thread.sleep(retryInterval);
+        } catch (InterruptedException ie) {
+          LOG.warn("Client retry sleep interrupted! ");
+        }
+      }
+      throw new RuntimeException("Failed to connect to timeline server. "
+          + "Connection retries limit exceeded. "
+          + "The posted timeline event may be missing");
+    };
+
+    private void logException(Exception e, int leftRetries) {
+      if (leftRetries > 0) {
+        LOG.info(
+            "Exception caught by TimelineClientConnectionRetry," + " will try "
+                + leftRetries + " more time(s).\nMessage: " + e.getMessage());
+      } else {
+        // note that maxRetries may be -1 at the very beginning
+        LOG.info("ConnectionException caught by TimelineClientConnectionRetry,"
+            + " will keep retrying.\nMessage: " + e.getMessage());
+      }
+    }
+  }
+
+  private static class TimelineJerseyRetryFilter extends ClientFilter {
+    private TimelineClientConnectionRetry connectionRetry;
+
+    public TimelineJerseyRetryFilter(
+        TimelineClientConnectionRetry connectionRetry) {
+      this.connectionRetry = connectionRetry;
+    }
+
+    @Override
+    public ClientResponse handle(final ClientRequest cr)
+        throws ClientHandlerException {
+      // Set up the retry operation
+      TimelineClientRetryOp jerseyRetryOp = new TimelineClientRetryOp() {
+        @Override
+        public Object run() {
+          // Try pass the request, if fail, keep retrying
+          return getNext().handle(cr);
+        }
+
+        @Override
+        public boolean shouldRetryOn(Exception e) {
+          // Only retry on connection exceptions
+          return (e instanceof ClientHandlerException)
+              && (e.getCause() instanceof ConnectException
+                  || e.getCause() instanceof SocketTimeoutException);
+        }
+      };
+      try {
+        return (ClientResponse) connectionRetry.retryOn(jerseyRetryOp);
+      } catch (IOException e) {
+        throw new ClientHandlerException(
+            "Jersey retry failed!\nMessage: " + e.getMessage());
+      }
+    }
+  }
+
+  @Private
+  @VisibleForTesting
+  public static class TimelineClientRetryOpForOperateDelegationToken
+      extends TimelineClientRetryOp {
+
+    private final UserGroupInformation authUgi;
+    private final PrivilegedExceptionAction<?> action;
+
+    public TimelineClientRetryOpForOperateDelegationToken(
+        UserGroupInformation authUgi, PrivilegedExceptionAction<?> action) {
+      this.authUgi = authUgi;
+      this.action = action;
+    }
+
+    @Override
+    public Object run() throws IOException {
+      // Try pass the request, if fail, keep retrying
+      authUgi.checkTGTAndReloginFromKeytab();
+      try {
+        return authUgi.doAs(action);
+      } catch (UndeclaredThrowableException e) {
+        throw new IOException(e.getCause());
+      } catch (InterruptedException e) {
+        throw new IOException(e);
+      }
+    }
+
+    @Override
+    public boolean shouldRetryOn(Exception e) {
+      // retry on connection exceptions
+      // and SocketTimeoutException
+      return (e instanceof ConnectException
+          || e instanceof SocketTimeoutException);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
new file mode 100644
index 0000000..cef7e5f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
@@ -0,0 +1,459 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.client.api.impl;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.core.util.MultivaluedMapImpl;
+
+/**
+ * Implementation of timeline v2 client interface.
+ *
+ */
+public class TimelineV2ClientImpl extends TimelineV2Client {
+  private static final Log LOG = LogFactory.getLog(TimelineV2ClientImpl.class);
+
+  private static final String RESOURCE_URI_STR_V2 = "/ws/v2/timeline/";
+
+  private TimelineEntityDispatcher entityDispatcher;
+  private volatile String timelineServiceAddress;
+
+  // Retry parameters for identifying new timeline service
+  // TODO consider to merge with connection retry
+  private int maxServiceRetries;
+  private long serviceRetryInterval;
+
+  private TimelineConnector connector;
+
+  private ApplicationId contextAppId;
+
+  public TimelineV2ClientImpl(ApplicationId appId) {
+    super(TimelineV2ClientImpl.class.getName());
+    this.contextAppId = appId;
+  }
+
+  public ApplicationId getContextAppId() {
+    return contextAppId;
+  }
+
+  protected void serviceInit(Configuration conf) throws Exception {
+    if (!YarnConfiguration.timelineServiceEnabled(conf)
+        || (int) YarnConfiguration.getTimelineServiceVersion(conf) != 2) {
+      throw new IOException("Timeline V2 client is not properly configured. "
+          + "Either timeline service is not enabled or version is not set to"
+          + " 2");
+    }
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    UserGroupInformation realUgi = ugi.getRealUser();
+    String doAsUser = null;
+    UserGroupInformation authUgi = null;
+    if (realUgi != null) {
+      authUgi = realUgi;
+      doAsUser = ugi.getShortUserName();
+    } else {
+      authUgi = ugi;
+      doAsUser = null;
+    }
+
+    // TODO need to add/cleanup filter retry later for ATSV2. similar to V1
+    DelegationTokenAuthenticatedURL.Token token =
+        new DelegationTokenAuthenticatedURL.Token();
+    connector = new TimelineConnector(false, authUgi, doAsUser, token);
+    addIfService(connector);
+
+    // new version need to auto discovery (with retry till ATS v2 address is
+    // got).
+    maxServiceRetries =
+        conf.getInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
+            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES);
+    serviceRetryInterval = conf.getLong(
+        YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
+        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS);
+    entityDispatcher = new TimelineEntityDispatcher(conf);
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    entityDispatcher.start();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    entityDispatcher.stop();
+    super.serviceStop();
+  }
+
+  @Override
+  public void putEntities(TimelineEntity... entities)
+      throws IOException, YarnException {
+    entityDispatcher.dispatchEntities(true, entities);
+  }
+
+  @Override
+  public void putEntitiesAsync(TimelineEntity... entities)
+      throws IOException, YarnException {
+    entityDispatcher.dispatchEntities(false, entities);
+  }
+
+  @Override
+  public void setTimelineServiceAddress(String address) {
+    this.timelineServiceAddress = address;
+  }
+
+  @Private
+  protected void putObjects(String path, MultivaluedMap<String, String> params,
+      Object obj) throws IOException, YarnException {
+
+    int retries = verifyRestEndPointAvailable();
+
+    // timelineServiceAddress could be stale, add retry logic here.
+    boolean needRetry = true;
+    while (needRetry) {
+      try {
+        URI uri = TimelineConnector.constructResURI(getConfig(),
+            timelineServiceAddress, RESOURCE_URI_STR_V2);
+        putObjects(uri, path, params, obj);
+        needRetry = false;
+      } catch (IOException e) {
+        // handle exception for timelineServiceAddress being updated.
+        checkRetryWithSleep(retries, e);
+        retries--;
+      }
+    }
+  }
+
+  /**
+   * Check if reaching to maximum of retries.
+   *
+   * @param retries
+   * @param e
+   */
+  private void checkRetryWithSleep(int retries, IOException e)
+      throws YarnException, IOException {
+    if (retries > 0) {
+      try {
+        Thread.sleep(this.serviceRetryInterval);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        throw new YarnException("Interrupted while retrying to connect to ATS");
+      }
+    } else {
+      StringBuilder msg =
+          new StringBuilder("TimelineClient has reached to max retry times : ");
+      msg.append(this.maxServiceRetries);
+      msg.append(" for service address: ");
+      msg.append(timelineServiceAddress);
+      LOG.error(msg.toString());
+      throw new IOException(msg.toString(), e);
+    }
+  }
+
+  protected void putObjects(URI base, String path,
+      MultivaluedMap<String, String> params, Object obj)
+      throws IOException, YarnException {
+    ClientResponse resp;
+    try {
+      resp = connector.getClient().resource(base).path(path).queryParams(params)
+          .accept(MediaType.APPLICATION_JSON).type(MediaType.APPLICATION_JSON)
+          .put(ClientResponse.class, obj);
+    } catch (RuntimeException re) {
+      // runtime exception is expected if the client cannot connect the server
+      String msg = "Failed to get the response from the timeline server.";
+      LOG.error(msg, re);
+      throw new IOException(re);
+    }
+    if (resp == null || resp.getStatusInfo()
+        .getStatusCode() != ClientResponse.Status.OK.getStatusCode()) {
+      String msg =
+          "Response from the timeline server is " + ((resp == null) ? "null"
+              : "not successful," + " HTTP error code: " + resp.getStatus()
+                  + ", Server response:\n" + resp.getEntity(String.class));
+      LOG.error(msg);
+      throw new YarnException(msg);
+    }
+  }
+
+  private int verifyRestEndPointAvailable() throws YarnException {
+    // timelineServiceAddress could haven't be initialized yet
+    // or stale (only for new timeline service)
+    int retries = pollTimelineServiceAddress(this.maxServiceRetries);
+    if (timelineServiceAddress == null) {
+      String errMessage = "TimelineClient has reached to max retry times : "
+          + this.maxServiceRetries
+          + ", but failed to fetch timeline service address. Please verify"
+          + " Timeline Auxiliary Service is configured in all the NMs";
+      LOG.error(errMessage);
+      throw new YarnException(errMessage);
+    }
+    return retries;
+  }
+
+  /**
+   * Poll TimelineServiceAddress for maximum of retries times if it is null.
+   *
+   * @param retries
+   * @return the left retry times
+   * @throws IOException
+   */
+  private int pollTimelineServiceAddress(int retries) throws YarnException {
+    while (timelineServiceAddress == null && retries > 0) {
+      try {
+        Thread.sleep(this.serviceRetryInterval);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new YarnException("Interrupted while trying to connect ATS");
+      }
+      retries--;
+    }
+    return retries;
+  }
+
+  private final class EntitiesHolder extends FutureTask<Void> {
+    private final TimelineEntities entities;
+    private final boolean isSync;
+
+    EntitiesHolder(final TimelineEntities entities, final boolean isSync) {
+      super(new Callable<Void>() {
+        // publishEntities()
+        public Void call() throws Exception {
+          MultivaluedMap<String, String> params = new MultivaluedMapImpl();
+          params.add("appid", getContextAppId().toString());
+          params.add("async", Boolean.toString(!isSync));
+          putObjects("entities", params, entities);
+          return null;
+        }
+      });
+      this.entities = entities;
+      this.isSync = isSync;
+    }
+
+    public boolean isSync() {
+      return isSync;
+    }
+
+    public TimelineEntities getEntities() {
+      return entities;
+    }
+  }
+
+  /**
+   * This class is responsible for collecting the timeline entities and
+   * publishing them in async.
+   */
+  private class TimelineEntityDispatcher {
+    /**
+     * Time period for which the timelineclient will wait for draining after
+     * stop.
+     */
+    private static final long DRAIN_TIME_PERIOD = 2000L;
+
+    private int numberOfAsyncsToMerge;
+    private final BlockingQueue<EntitiesHolder> timelineEntityQueue;
+    private ExecutorService executor;
+
+    TimelineEntityDispatcher(Configuration conf) {
+      timelineEntityQueue = new LinkedBlockingQueue<EntitiesHolder>();
+      numberOfAsyncsToMerge =
+          conf.getInt(YarnConfiguration.NUMBER_OF_ASYNC_ENTITIES_TO_MERGE,
+              YarnConfiguration.DEFAULT_NUMBER_OF_ASYNC_ENTITIES_TO_MERGE);
+    }
+
+    Runnable createRunnable() {
+      return new Runnable() {
+        @Override
+        public void run() {
+          try {
+            EntitiesHolder entitiesHolder;
+            while (!Thread.currentThread().isInterrupted()) {
+              // Merge all the async calls and make one push, but if its sync
+              // call push immediately
+              try {
+                entitiesHolder = timelineEntityQueue.take();
+              } catch (InterruptedException ie) {
+                LOG.info("Timeline dispatcher thread was interrupted ");
+                Thread.currentThread().interrupt();
+                return;
+              }
+              if (entitiesHolder != null) {
+                publishWithoutBlockingOnQueue(entitiesHolder);
+              }
+            }
+          } finally {
+            if (!timelineEntityQueue.isEmpty()) {
+              LOG.info("Yet to publish " + timelineEntityQueue.size()
+                  + " timelineEntities, draining them now. ");
+            }
+            // Try to drain the remaining entities to be published @ the max for
+            // 2 seconds
+            long timeTillweDrain =
+                System.currentTimeMillis() + DRAIN_TIME_PERIOD;
+            while (!timelineEntityQueue.isEmpty()) {
+              publishWithoutBlockingOnQueue(timelineEntityQueue.poll());
+              if (System.currentTimeMillis() > timeTillweDrain) {
+                // time elapsed stop publishing further....
+                if (!timelineEntityQueue.isEmpty()) {
+                  LOG.warn("Time to drain elapsed! Remaining "
+                      + timelineEntityQueue.size() + "timelineEntities will not"
+                      + " be published");
+                  // if some entities were not drained then we need interrupt
+                  // the threads which had put sync EntityHolders to the queue.
+                  EntitiesHolder nextEntityInTheQueue = null;
+                  while ((nextEntityInTheQueue =
+                      timelineEntityQueue.poll()) != null) {
+                    nextEntityInTheQueue.cancel(true);
+                  }
+                }
+                break;
+              }
+            }
+          }
+        }
+
+        /**
+         * Publishes the given EntitiesHolder and return immediately if sync
+         * call, else tries to fetch the EntitiesHolder from the queue in non
+         * blocking fashion and collate the Entities if possible before
+         * publishing through REST.
+         *
+         * @param entitiesHolder
+         */
+        private void publishWithoutBlockingOnQueue(
+            EntitiesHolder entitiesHolder) {
+          if (entitiesHolder.isSync()) {
+            entitiesHolder.run();
+            return;
+          }
+          int count = 1;
+          while (true) {
+            // loop till we find a sync put Entities or there is nothing
+            // to take
+            EntitiesHolder nextEntityInTheQueue = timelineEntityQueue.poll();
+            if (nextEntityInTheQueue == null) {
+              // Nothing in the queue just publish and get back to the
+              // blocked wait state
+              entitiesHolder.run();
+              break;
+            } else if (nextEntityInTheQueue.isSync()) {
+              // flush all the prev async entities first
+              entitiesHolder.run();
+              // and then flush the sync entity
+              nextEntityInTheQueue.run();
+              break;
+            } else {
+              // append all async entities together and then flush
+              entitiesHolder.getEntities().addEntities(
+                  nextEntityInTheQueue.getEntities().getEntities());
+              count++;
+              if (count == numberOfAsyncsToMerge) {
+                // Flush the entities if the number of the async
+                // putEntites merged reaches the desired limit. To avoid
+                // collecting multiple entities and delaying for a long
+                // time.
+                entitiesHolder.run();
+                break;
+              }
+            }
+          }
+        }
+      };
+    }
+
+    public void dispatchEntities(boolean sync,
+        TimelineEntity[] entitiesTobePublished) throws YarnException {
+      if (executor.isShutdown()) {
+        throw new YarnException("Timeline client is in the process of stopping,"
+            + " not accepting any more TimelineEntities");
+      }
+
+      // wrap all TimelineEntity into TimelineEntities object
+      TimelineEntities entities = new TimelineEntities();
+      for (TimelineEntity entity : entitiesTobePublished) {
+        entities.addEntity(entity);
+      }
+
+      // created a holder and place it in queue
+      EntitiesHolder entitiesHolder = new EntitiesHolder(entities, sync);
+      try {
+        timelineEntityQueue.put(entitiesHolder);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new YarnException(
+            "Failed while adding entity to the queue for publishing", e);
+      }
+
+      if (sync) {
+        // In sync call we need to wait till its published and if any error then
+        // throw it back
+        try {
+          entitiesHolder.get();
+        } catch (ExecutionException e) {
+          throw new YarnException("Failed while publishing entity",
+              e.getCause());
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new YarnException("Interrupted while publishing entity", e);
+        }
+      }
+    }
+
+    public void start() {
+      executor = Executors.newSingleThreadExecutor();
+      executor.execute(createRunnable());
+    }
+
+    public void stop() {
+      LOG.info("Stopping TimelineClient.");
+      executor.shutdownNow();
+      try {
+        executor.awaitTermination(DRAIN_TIME_PERIOD, TimeUnit.MILLISECONDS);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        e.printStackTrace();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
index bfc7cbd..f42c078 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
@@ -215,11 +215,11 @@ public class TestTimelineClient {
           + "Timeline server should be off to run this test. ");
     } catch (RuntimeException ce) {
       Assert.assertTrue(
-        "Handler exception for reason other than retry: " + ce.getMessage(),
-        ce.getMessage().contains("Connection retries limit exceeded"));
+          "Handler exception for reason other than retry: " + ce.getMessage(),
+          ce.getMessage().contains("Connection retries limit exceeded"));
       // we would expect this exception here, check if the client has retried
-      Assert.assertTrue("Retry filter didn't perform any retries! ", client
-        .connectionRetry.getRetired());
+      Assert.assertTrue("Retry filter didn't perform any retries! ",
+          client.connector.connectionRetry.getRetired());
     }
   }
 
@@ -318,7 +318,7 @@ public class TestTimelineClient {
             .getMessage().contains("Connection retries limit exceeded"));
     // we would expect this exception here, check if the client has retried
     Assert.assertTrue("Retry filter didn't perform any retries! ",
-        client.connectionRetry.getRetired());
+        client.connector.connectionRetry.getRetired());
   }
 
   public static ClientResponse mockEntityClientResponse(
@@ -419,17 +419,26 @@ public class TestTimelineClient {
   private TimelineClientImpl createTimelineClientFakeTimelineClientRetryOp(
       YarnConfiguration conf) {
     TimelineClientImpl client = new TimelineClientImpl() {
-
       @Override
-      public TimelineClientRetryOp
-          createTimelineClientRetryOpForOperateDelegationToken(
-              final PrivilegedExceptionAction<?> action) throws IOException {
-        TimelineClientRetryOpForOperateDelegationToken op =
-            spy(new TimelineClientRetryOpForOperateDelegationToken(
-            UserGroupInformation.getCurrentUser(), action));
-        doThrow(new SocketTimeoutException("Test socketTimeoutException"))
-            .when(op).run();
-        return op;
+      protected TimelineConnector createTimelineConnector() {
+        TimelineConnector connector =
+            new TimelineConnector(true, authUgi, doAsUser, token) {
+              @Override
+              public TimelineClientRetryOp
+                createRetryOpForOperateDelegationToken(
+                  final PrivilegedExceptionAction<?> action)
+                  throws IOException {
+                TimelineClientRetryOpForOperateDelegationToken op =
+                    spy(new TimelineClientRetryOpForOperateDelegationToken(
+                        UserGroupInformation.getCurrentUser(), action));
+                doThrow(
+                    new SocketTimeoutException("Test socketTimeoutException"))
+                        .when(op).run();
+                return op;
+              }
+            };
+        addIfService(connector);
+        return connector;
       }
     };
     client.init(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java
index 5813340..c5b02fd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java
@@ -50,7 +50,7 @@ public class TestTimelineClientV2Impl {
   public void setup() {
     conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
-    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 1.0f);
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
     conf.setInt(YarnConfiguration.NUMBER_OF_ASYNC_ENTITIES_TO_MERGE, 3);
     if (!currTestName.getMethodName()
         .contains("testRetryOnConnectionFailure")) {
@@ -71,7 +71,7 @@ public class TestTimelineClientV2Impl {
   }
 
   private class TestV2TimelineClientForExceptionHandling
-      extends TimelineClientImpl {
+      extends TimelineV2ClientImpl {
     public TestV2TimelineClientForExceptionHandling(ApplicationId id) {
       super(id);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.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/timelineservice/NMTimelinePublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
index 8994582..ce2c656 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
-import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -76,7 +76,7 @@ public class NMTimelinePublisher extends CompositeService {
 
   private String httpAddress;
 
-  private final Map<ApplicationId, TimelineClient> appToClientMap;
+  private final Map<ApplicationId, TimelineV2Client> appToClientMap;
 
   public NMTimelinePublisher(Context context) {
     super(NMTimelinePublisher.class.getName());
@@ -102,7 +102,7 @@ public class NMTimelinePublisher extends CompositeService {
   }
 
   @VisibleForTesting
-  Map<ApplicationId, TimelineClient> getAppToClientMap() {
+  Map<ApplicationId, TimelineV2Client> getAppToClientMap() {
     return appToClientMap;
   }
 
@@ -145,7 +145,7 @@ public class NMTimelinePublisher extends CompositeService {
       try {
         // no need to put it as part of publisher as timeline client already has
         // Queuing concept
-        TimelineClient timelineClient = getTimelineClient(appId);
+        TimelineV2Client timelineClient = getTimelineClient(appId);
         if (timelineClient != null) {
           timelineClient.putEntitiesAsync(entity);
         } else {
@@ -234,7 +234,7 @@ public class NMTimelinePublisher extends CompositeService {
     try {
       // no need to put it as part of publisher as timeline client already has
       // Queuing concept
-      TimelineClient timelineClient = getTimelineClient(appId);
+      TimelineV2Client timelineClient = getTimelineClient(appId);
       if (timelineClient != null) {
         timelineClient.putEntitiesAsync(entity);
       } else {
@@ -265,7 +265,7 @@ public class NMTimelinePublisher extends CompositeService {
         LOG.debug("Publishing the entity " + entity + ", JSON-style content: "
             + TimelineUtils.dumpTimelineRecordtoJSON(entity));
       }
-      TimelineClient timelineClient = getTimelineClient(appId);
+      TimelineV2Client timelineClient = getTimelineClient(appId);
       if (timelineClient != null) {
         timelineClient.putEntities(entity);
       } else {
@@ -382,8 +382,8 @@ public class NMTimelinePublisher extends CompositeService {
 
   public void createTimelineClient(ApplicationId appId) {
     if (!appToClientMap.containsKey(appId)) {
-      TimelineClient timelineClient =
-          TimelineClient.createTimelineClient(appId);
+      TimelineV2Client timelineClient =
+          TimelineV2Client.createTimelineClient(appId);
       timelineClient.init(getConfig());
       timelineClient.start();
       appToClientMap.put(appId, timelineClient);
@@ -391,7 +391,7 @@ public class NMTimelinePublisher extends CompositeService {
   }
 
   public void stopTimelineClient(ApplicationId appId) {
-    TimelineClient client = appToClientMap.remove(appId);
+    TimelineV2Client client = appToClientMap.remove(appId);
     if (client != null) {
       client.stop();
     }
@@ -399,13 +399,13 @@ public class NMTimelinePublisher extends CompositeService {
 
   public void setTimelineServiceAddress(ApplicationId appId,
       String collectorAddr) {
-    TimelineClient client = appToClientMap.get(appId);
+    TimelineV2Client client = appToClientMap.get(appId);
     if (client != null) {
       client.setTimelineServiceAddress(collectorAddr);
     }
   }
 
-  private TimelineClient getTimelineClient(ApplicationId appId) {
+  private TimelineV2Client getTimelineClient(ApplicationId appId) {
     return appToClientMap.get(appId);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.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/timelineservice/TestNMTimelinePublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java
index ae9397a..e116122 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
-import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;
+import org.apache.hadoop.yarn.client.api.impl.TimelineV2ClientImpl;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
@@ -50,7 +50,7 @@ public class TestNMTimelinePublisher {
   public void testContainerResourceUsage() {
     Context context = mock(Context.class);
     @SuppressWarnings("unchecked")
-    final DummyTimelineClient timelineClient = new DummyTimelineClient();
+    final DummyTimelineClient timelineClient = new DummyTimelineClient(null);
     when(context.getNodeId()).thenReturn(NodeId.newInstance("localhost", 0));
     when(context.getHttpPort()).thenReturn(0);
     NMTimelinePublisher publisher = new NMTimelinePublisher(context) {
@@ -137,7 +137,11 @@ public class TestNMTimelinePublisher {
     }
   }
 
-  protected static class DummyTimelineClient extends TimelineClientImpl {
+  protected static class DummyTimelineClient extends TimelineV2ClientImpl {
+    public DummyTimelineClient(ApplicationId appId) {
+      super(appId);
+    }
+
     private TimelineEntity[] lastPublishedEntities;
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java
index 3ec222f..07058f6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.QueueEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.api.records.timelineservice.UserEntity;
-import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol;
@@ -96,8 +96,8 @@ public class TestTimelineServiceClientIntegration {
 
   @Test
   public void testPutEntities() throws Exception {
-    TimelineClient client =
-        TimelineClient.createTimelineClient(ApplicationId.newInstance(0, 1));
+    TimelineV2Client client =
+        TimelineV2Client.createTimelineClient(ApplicationId.newInstance(0, 1));
     try {
       // set the timeline service address manually
       client.setTimelineServiceAddress(
@@ -123,8 +123,8 @@ public class TestTimelineServiceClientIntegration {
   @Test
   public void testPutExtendedEntities() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(0, 1);
-    TimelineClient client =
-        TimelineClient.createTimelineClient(appId);
+    TimelineV2Client client =
+        TimelineV2Client.createTimelineClient(appId);
     try {
       // set the timeline service address manually
       client.setTimelineServiceAddress(


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


[10/50] [abbrv] hadoop git commit: HADOOP-13942. Update checkstyle and checkstyle plugin version to handle indentation of JDK8 Lambdas.

Posted by xg...@apache.org.
HADOOP-13942. Update checkstyle and checkstyle plugin version to handle indentation of JDK8 Lambdas.


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

Branch: refs/heads/YARN-5734
Commit: 1e11080b7825a2d0bafce91432009f585b7b5d21
Parents: fbc0c2b
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Feb 15 16:33:30 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Feb 15 16:35:08 2017 +0900

----------------------------------------------------------------------
 hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml | 4 +---
 pom.xml                                                         | 4 ++--
 2 files changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e11080b/hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml
----------------------------------------------------------------------
diff --git a/hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml b/hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml
index 851b57d..1b968ae 100644
--- a/hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml
+++ b/hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml
@@ -123,9 +123,7 @@
 
         <!-- Checks for Size Violations.                    -->
         <!-- See http://checkstyle.sf.net/config_sizes.html -->
-        <module name="LineLength">
-          <property name="ignorePattern" value="^(package|import) .*"/>
-        </module>
+        <module name="LineLength"/>
         <module name="MethodLength"/>
         <module name="ParameterNumber"/>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e11080b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2ca27c1..3eeba1e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -107,8 +107,8 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
     <maven-clover2-plugin.version>3.3.0</maven-clover2-plugin.version>
     <maven-bundle-plugin.version>2.5.0</maven-bundle-plugin.version>
     <lifecycle-mapping.version>1.0.0</lifecycle-mapping.version>
-    <maven-checkstyle-plugin.version>2.15</maven-checkstyle-plugin.version>
-    <checkstyle.version>6.6</checkstyle.version>
+    <maven-checkstyle-plugin.version>2.17</maven-checkstyle-plugin.version>
+    <checkstyle.version>7.5.1</checkstyle.version>
     <dependency-check-maven.version>1.4.3</dependency-check-maven.version>
 
     <shell-executable>bash</shell-executable>


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


[43/50] [abbrv] hadoop git commit: HDFS-11177. 'storagepolicies -getStoragePolicy' command should accept URI based path. (Contributed by Surendra Singh Lilhore)

Posted by xg...@apache.org.
HDFS-11177. 'storagepolicies -getStoragePolicy' command should accept URI based path. (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/48040506
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/48040506
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/48040506

Branch: refs/heads/YARN-5734
Commit: 480405063063f564ae0cdb34e0757ac3990569aa
Parents: 6ba61d2
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Feb 21 18:13:19 2017 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Tue Feb 21 18:13:19 2017 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/tools/AdminHelper.java   | 12 ++++++++++
 .../hadoop/hdfs/tools/StoragePolicyAdmin.java   | 23 +++++++++++---------
 .../hdfs/tools/TestStoragePolicyCommands.java   | 14 +++++++-----
 3 files changed, 33 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/48040506/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/AdminHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/AdminHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/AdminHelper.java
index 153fb36..8bab550 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/AdminHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/AdminHelper.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.tools;
 
 import com.google.common.base.Preconditions;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -26,6 +27,7 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.tools.TableListing;
 
 import java.io.IOException;
+import java.net.URI;
 import java.util.List;
 
 /**
@@ -48,6 +50,16 @@ public class AdminHelper {
     return (DistributedFileSystem)fs;
   }
 
+  static DistributedFileSystem getDFS(URI uri, Configuration conf)
+      throws IOException {
+    FileSystem fs = FileSystem.get(uri, conf);
+    if (!(fs instanceof DistributedFileSystem)) {
+      throw new IllegalArgumentException("FileSystem " + fs.getUri()
+          + " is not an HDFS file system");
+    }
+    return (DistributedFileSystem) fs;
+  }
+
   /**
    * NN exceptions contain the stack trace as part of the exception message.
    * When it's a known error, pretty-print the error and squish the stack trace.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48040506/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
index d99b88a..4e4f018 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
@@ -148,9 +148,11 @@ public class StoragePolicyAdmin extends Configured implements Tool {
         return 1;
       }
 
-      final DistributedFileSystem dfs = AdminHelper.getDFS(conf);
+      Path p = new Path(path);
+      final DistributedFileSystem dfs = AdminHelper.getDFS(p.toUri(), conf);
       try {
-        HdfsFileStatus status = dfs.getClient().getFileInfo(path);
+        HdfsFileStatus status = dfs.getClient().getFileInfo(
+            Path.getPathWithoutSchemeAndAuthority(p).toString());
         if (status == null) {
           System.err.println("File/Directory does not exist: " + path);
           return 2;
@@ -161,9 +163,9 @@ public class StoragePolicyAdmin extends Configured implements Tool {
           return 0;
         }
         Collection<BlockStoragePolicy> policies = dfs.getAllStoragePolicies();
-        for (BlockStoragePolicy p : policies) {
-          if (p.getId() == storagePolicyId) {
-            System.out.println("The storage policy of " + path + ":\n" + p);
+        for (BlockStoragePolicy policy : policies) {
+          if (policy.getId() == storagePolicyId) {
+            System.out.println("The storage policy of " + path + ":\n" + policy);
             return 0;
           }
         }
@@ -215,10 +217,10 @@ public class StoragePolicyAdmin extends Configured implements Tool {
             getLongUsage());
         return 1;
       }
-
-      final DistributedFileSystem dfs = AdminHelper.getDFS(conf);
+      Path p = new Path(path);
+      final DistributedFileSystem dfs = AdminHelper.getDFS(p.toUri(), conf);
       try {
-        dfs.setStoragePolicy(new Path(path), policyName);
+        dfs.setStoragePolicy(p, policyName);
         System.out.println("Set storage policy " + policyName + " on " + path);
       } catch (Exception e) {
         System.err.println(AdminHelper.prettifyException(e));
@@ -261,9 +263,10 @@ public class StoragePolicyAdmin extends Configured implements Tool {
         return 1;
       }
 
-      final DistributedFileSystem dfs = AdminHelper.getDFS(conf);
+      Path p = new Path(path);
+      final DistributedFileSystem dfs = AdminHelper.getDFS(p.toUri(), conf);
       try {
-        dfs.unsetStoragePolicy(new Path(path));
+        dfs.unsetStoragePolicy(p);
         System.out.println("Unset storage policy from " + path);
       } catch (Exception e) {
         System.err.println(AdminHelper.prettifyException(e));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48040506/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestStoragePolicyCommands.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestStoragePolicyCommands.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestStoragePolicyCommands.java
index 63f8484..149dabb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestStoragePolicyCommands.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestStoragePolicyCommands.java
@@ -74,8 +74,9 @@ public class TestStoragePolicyCommands {
      * test: set storage policy
      */
     final StoragePolicyAdmin admin = new StoragePolicyAdmin(conf);
-    DFSTestUtil.toolRun(admin, "-setStoragePolicy -path /foo -policy WARM", 0,
-        "Set storage policy WARM on " + foo.toString());
+    DFSTestUtil.toolRun(admin, "-setStoragePolicy -path " + fs.getUri()
+        + "/foo -policy WARM", 0, "Set storage policy WARM on " + fs.getUri()
+        + "/foo");
     DFSTestUtil.toolRun(admin, "-setStoragePolicy -path /foo/bar -policy COLD",
         0, "Set storage policy COLD on " + bar.toString());
     DFSTestUtil.toolRun(admin, "-setStoragePolicy -path /foo/bar/wow -policy HOT",
@@ -91,8 +92,9 @@ public class TestStoragePolicyCommands {
     final BlockStoragePolicy warm = suite.getPolicy("WARM");
     final BlockStoragePolicy cold = suite.getPolicy("COLD");
     final BlockStoragePolicy hot = suite.getPolicy("HOT");
-    DFSTestUtil.toolRun(admin, "-getStoragePolicy -path /foo", 0,
-        "The storage policy of " + foo.toString() + ":\n" + warm);
+    DFSTestUtil.toolRun(admin, "-getStoragePolicy -path " + fs.getUri()
+        + "/foo", 0, "The storage policy of " + fs.getUri() + "/foo:\n"
+        + warm);
     DFSTestUtil.toolRun(admin, "-getStoragePolicy -path /foo/bar", 0,
         "The storage policy of " + bar.toString() + ":\n" + cold);
     DFSTestUtil.toolRun(admin, "-getStoragePolicy -path /foo/bar/wow", 0,
@@ -103,8 +105,8 @@ public class TestStoragePolicyCommands {
     /*
      * test: unset storage policy
      */
-    DFSTestUtil.toolRun(admin, "-unsetStoragePolicy -path /foo", 0,
-        "Unset storage policy from " + foo.toString());
+    DFSTestUtil.toolRun(admin, "-unsetStoragePolicy -path " + fs.getUri()
+        + "/foo", 0, "Unset storage policy from " + fs.getUri() + "/foo");
     DFSTestUtil.toolRun(admin, "-unsetStoragePolicy -path /foo/bar", 0,
         "Unset storage policy from " + bar.toString());
     DFSTestUtil.toolRun(admin, "-unsetStoragePolicy -path /foo/bar/wow", 0,


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


[19/50] [abbrv] hadoop git commit: HDFS-11375. Display the volume storage type in datanode UI. Contributed by Surendra Singh Lilhore

Posted by xg...@apache.org.
HDFS-11375. Display the volume storage type in datanode UI. 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/0741dd3b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0741dd3b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0741dd3b

Branch: refs/heads/YARN-5734
Commit: 0741dd3b9abdeb65bb783c1a8b01f078c4bdba17
Parents: 627da6f
Author: Mingliang Liu <li...@apache.org>
Authored: Wed Feb 15 11:37:26 2017 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Wed Feb 15 11:37:26 2017 -0800

----------------------------------------------------------------------
 .../hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java | 3 +++
 .../hadoop-hdfs/src/main/webapps/datanode/datanode.html           | 2 ++
 2 files changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0741dd3b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index d1f8f05..6d00d75 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -2606,6 +2606,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     final long reservedSpaceForReplicas; // size of space reserved RBW or
                                     // re-replication
     final long numBlocks;
+    final StorageType storageType;
 
     VolumeInfo(FsVolumeImpl v, long usedSpace, long freeSpace) {
       this.directory = v.toString();
@@ -2614,6 +2615,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       this.reservedSpace = v.getReserved();
       this.reservedSpaceForReplicas = v.getReservedForReplicas();
       this.numBlocks = v.getNumBlocks();
+      this.storageType = v.getStorageType();
     }
   }  
 
@@ -2649,6 +2651,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       innerInfo.put("reservedSpace", v.reservedSpace);
       innerInfo.put("reservedSpaceForReplicas", v.reservedSpaceForReplicas);
       innerInfo.put("numBlocks", v.numBlocks);
+      innerInfo.put("storageType", v.storageType);
       info.put(v.directory, innerInfo);
     }
     return info;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0741dd3b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/datanode.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/datanode.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/datanode.html
index b35a0a7..e474ab5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/datanode.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/datanode.html
@@ -100,6 +100,7 @@
   <thead>
     <tr>
       <th>Directory</th>
+      <th>StorageType</th>
       <th>Capacity Used</th>
       <th>Capacity Left</th>
       <th>Capacity Reserved</th>
@@ -110,6 +111,7 @@
   {#dn.VolumeInfo}
     <tr>
       <td>{name}</td>
+      <td>{storageType}</td>
       <td>{usedSpace|fmt_bytes}</td>
       <td>{freeSpace|fmt_bytes}</td>
       <td>{reservedSpace|fmt_bytes}</td>


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


[06/50] [abbrv] hadoop git commit: YARN-6061. Add an UncaughtExceptionHandler for critical threads in RM. (Yufei Gu via kasha)

Posted by xg...@apache.org.
YARN-6061. Add an UncaughtExceptionHandler for critical threads in RM. (Yufei Gu via kasha)


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

Branch: refs/heads/YARN-5734
Commit: 652679aa8ad6f9e61b8ed8e2b04b3e0332025e94
Parents: aaf2713
Author: Karthik Kambatla <ka...@cloudera.com>
Authored: Tue Feb 14 13:39:34 2017 -0800
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Tue Feb 14 13:39:41 2017 -0800

----------------------------------------------------------------------
 .../hadoop/yarn/client/TestRMFailover.java      | 100 ++++++++++++++++++-
 .../yarn/server/resourcemanager/RMContext.java  |   2 +
 .../server/resourcemanager/RMContextImpl.java   |  10 ++
 ...MCriticalThreadUncaughtExceptionHandler.java |  58 +++++++++++
 .../resourcemanager/RMFatalEventType.java       |   5 +-
 .../server/resourcemanager/ResourceManager.java |  65 +++++++++---
 .../resourcemanager/recovery/RMStateStore.java  |  13 +--
 .../DominantResourceFairnessPolicy.java         |   2 +-
 8 files changed, 226 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/652679aa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java
index b58a775..4bf6a78 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java
@@ -22,7 +22,10 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
 import static org.junit.Assert.fail;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
 
 import java.io.IOException;
 import java.net.HttpURLConnection;
@@ -37,14 +40,18 @@ import org.apache.hadoop.ha.ClientBaseWithFixes;
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.service.Service.STATE;
+import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.server.resourcemanager.AdminService;
 import org.apache.hadoop.yarn.server.resourcemanager.HATestUtil;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.RMCriticalThreadUncaughtExceptionHandler;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.webproxy.WebAppProxyServer;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
 import org.junit.After;
@@ -174,7 +181,7 @@ public class TestRMFailover extends ClientBaseWithFixes {
     // so it transitions to standby.
     ResourceManager rm = cluster.getResourceManager(
         cluster.getActiveRMIndex());
-    rm.handleTransitionToStandBy();
+    rm.handleTransitionToStandByInNewThread();
     int maxWaitingAttempts = 2000;
     while (maxWaitingAttempts-- > 0 ) {
       if (rm.getRMContext().getHAServiceState() == HAServiceState.STANDBY) {
@@ -349,4 +356,95 @@ public class TestRMFailover extends ClientBaseWithFixes {
     }
     return redirectUrl;
   }
+
+  /**
+   * Throw {@link RuntimeException} inside a thread of
+   * {@link ResourceManager} with HA enabled and check if the
+   * {@link ResourceManager} is transited to standby state.
+   *
+   * @throws InterruptedException if any
+   */
+  @Test
+  public void testUncaughtExceptionHandlerWithHAEnabled()
+      throws InterruptedException {
+    conf.set(YarnConfiguration.RM_CLUSTER_ID, "yarn-test-cluster");
+    conf.set(YarnConfiguration.RM_ZK_ADDRESS, hostPort);
+    cluster.init(conf);
+    cluster.start();
+    assertFalse("RM never turned active", -1 == cluster.getActiveRMIndex());
+
+    ResourceManager resourceManager = cluster.getResourceManager(
+        cluster.getActiveRMIndex());
+
+    final RMCriticalThreadUncaughtExceptionHandler exHandler =
+        new RMCriticalThreadUncaughtExceptionHandler(
+            resourceManager.getRMContext());
+
+    // Create a thread and throw a RTE inside it
+    final RuntimeException rte = new RuntimeException("TestRuntimeException");
+    final Thread testThread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        throw rte;
+      }
+    });
+    testThread.setName("TestThread");
+    testThread.setUncaughtExceptionHandler(exHandler);
+    testThread.start();
+    testThread.join();
+
+    int maxWaitingAttempts = 2000;
+    while (maxWaitingAttempts-- > 0) {
+      if (resourceManager.getRMContext().getHAServiceState()
+          == HAServiceState.STANDBY) {
+        break;
+      }
+      Thread.sleep(1);
+    }
+    assertFalse("RM didn't transition to Standby ", maxWaitingAttempts < 0);
+  }
+
+  /**
+   * Throw {@link RuntimeException} inside a thread of
+   * {@link ResourceManager} with HA disabled and check
+   * {@link RMCriticalThreadUncaughtExceptionHandler} instance.
+   *
+   * Used {@link ExitUtil} class to avoid jvm exit through
+   * {@code System.exit(-1)}.
+   *
+   * @throws InterruptedException if any
+   */
+  @Test
+  public void testUncaughtExceptionHandlerWithoutHA()
+      throws InterruptedException {
+    ExitUtil.disableSystemExit();
+
+    // Create a MockRM and start it
+    ResourceManager resourceManager = new MockRM();
+    ((AsyncDispatcher) resourceManager.getRMContext().getDispatcher()).start();
+    resourceManager.getRMContext().getStateStore().start();
+    resourceManager.getRMContext().getContainerTokenSecretManager().
+        rollMasterKey();
+
+    final RMCriticalThreadUncaughtExceptionHandler exHandler =
+        new RMCriticalThreadUncaughtExceptionHandler(
+            resourceManager.getRMContext());
+    final RMCriticalThreadUncaughtExceptionHandler spyRTEHandler =
+        spy(exHandler);
+
+    // Create a thread and throw a RTE inside it
+    final RuntimeException rte = new RuntimeException("TestRuntimeException");
+    final Thread testThread = new Thread(new Runnable() {
+      @Override public void run() {
+        throw rte;
+      }
+    });
+    testThread.setName("TestThread");
+    testThread.setUncaughtExceptionHandler(spyRTEHandler);
+    assertSame(spyRTEHandler, testThread.getUncaughtExceptionHandler());
+    testThread.start();
+    testThread.join();
+
+    verify(spyRTEHandler).uncaughtException(testThread, rte);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/652679aa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.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/RMContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
index 26ef5ac..ba6b915 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
@@ -156,4 +156,6 @@ public interface RMContext {
   RMAppLifetimeMonitor getRMAppLifetimeMonitor();
 
   String getHAZookeeperConnectionState();
+
+  ResourceManager getResourceManager();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/652679aa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index a452f95..fb160c4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -82,6 +82,7 @@ public class RMContextImpl implements RMContext {
 
   private final Object haServiceStateLock = new Object();
 
+  private ResourceManager resourceManager;
   /**
    * Default constructor. To be used in conjunction with setter methods for
    * individual fields.
@@ -522,4 +523,13 @@ public class RMContextImpl implements RMContext {
       return elector.getZookeeperConnectionState();
     }
   }
+
+  @Override
+  public ResourceManager getResourceManager() {
+    return resourceManager;
+  }
+
+  public void setResourceManager(ResourceManager rm) {
+    this.resourceManager = rm;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/652679aa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMCriticalThreadUncaughtExceptionHandler.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/RMCriticalThreadUncaughtExceptionHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMCriticalThreadUncaughtExceptionHandler.java
new file mode 100644
index 0000000..c5c6087
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMCriticalThreadUncaughtExceptionHandler.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager;
+
+import java.lang.Thread.UncaughtExceptionHandler;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.conf.HAUtil;
+
+/**
+ * This class either shuts down {@link ResourceManager} or transitions the
+ * {@link ResourceManager} to standby state if a critical thread throws an
+ * uncaught exception. It is intended to be installed by calling
+ * {@code setUncaughtExceptionHandler(Thread.UncaughtExceptionHandler)}
+ * in the thread entry point or after creation of threads.
+ */
+@Private
+public class RMCriticalThreadUncaughtExceptionHandler
+    implements UncaughtExceptionHandler {
+  private static final Log LOG = LogFactory.getLog(
+      RMCriticalThreadUncaughtExceptionHandler.class);
+  private RMContext rmContext;
+
+  public RMCriticalThreadUncaughtExceptionHandler(RMContext rmContext) {
+    this.rmContext = rmContext;
+  }
+
+  @Override
+  public void uncaughtException(Thread t, Throwable e) {
+    LOG.fatal("Critical thread " + t.getName() + " crashed!", e);
+
+    if (HAUtil.isHAEnabled(rmContext.getYarnConfiguration())) {
+      rmContext.getResourceManager().handleTransitionToStandByInNewThread();
+    } else {
+      rmContext.getDispatcher().getEventHandler().handle(
+          new RMFatalEvent(RMFatalEventType.CRITICAL_THREAD_CRASH,
+              new Exception(e)));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/652679aa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMFatalEventType.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/RMFatalEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMFatalEventType.java
index 87cc496..b6f6b3c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMFatalEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMFatalEventType.java
@@ -29,5 +29,8 @@ public enum RMFatalEventType {
   EMBEDDED_ELECTOR_FAILED,
 
   // Source <- Admin Service
-  TRANSITION_TO_ACTIVE_FAILED
+  TRANSITION_TO_ACTIVE_FAILED,
+
+  // Source <- Critical Thread Crash
+  CRITICAL_THREAD_CRASH
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/652679aa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.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/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index 886f4ab..58e4077 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -130,6 +130,7 @@ import java.security.PrivilegedExceptionAction;
 import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
  * The ResourceManager is the main class that is a set of components.
@@ -205,7 +206,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
   private Configuration conf;
 
   private UserGroupInformation rmLoginUGI;
-  
+
   public ResourceManager() {
     super("ResourceManager");
   }
@@ -232,7 +233,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
   protected void serviceInit(Configuration conf) throws Exception {
     this.conf = conf;
     this.rmContext = new RMContextImpl();
-    
+    rmContext.setResourceManager(this);
+
     this.configurationProvider =
         ConfigurationProviderFactory.getConfigurationProvider(conf);
     this.configurationProvider.init(this.conf);
@@ -564,6 +566,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
     private ResourceManager rm;
     private RMActiveServiceContext activeServiceContext;
     private boolean fromActive = false;
+    private StandByTransitionRunnable standByTransitionRunnable;
 
     RMActiveServices(ResourceManager rm) {
       super("RMActiveServices");
@@ -572,6 +575,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
 
     @Override
     protected void serviceInit(Configuration configuration) throws Exception {
+      standByTransitionRunnable = new StandByTransitionRunnable();
+
       activeServiceContext = new RMActiveServiceContext();
       rmContext.setActiveServiceContext(activeServiceContext);
 
@@ -819,19 +824,51 @@ public class ResourceManager extends CompositeService implements Recoverable {
     }
   }
 
-  public void handleTransitionToStandBy() {
-    if (rmContext.isHAEnabled()) {
-      try {
-        // Transition to standby and reinit active services
-        LOG.info("Transitioning RM to Standby mode");
-        transitionToStandby(true);
-        EmbeddedElector elector = rmContext.getLeaderElectorService();
-        if (elector != null) {
-          elector.rejoinElection();
+  /**
+   * Transition to standby state in a new thread. The transition operation is
+   * asynchronous to avoid deadlock caused by cyclic dependency.
+   */
+  public void handleTransitionToStandByInNewThread() {
+    Thread standByTransitionThread =
+        new Thread(activeServices.standByTransitionRunnable);
+    standByTransitionThread.setName("StandByTransitionThread");
+    standByTransitionThread.start();
+  }
+
+  /**
+   * The class to transition RM to standby state. The same
+   * {@link StandByTransitionRunnable} object could be used in multiple threads,
+   * but runs only once. That's because RM can go back to active state after
+   * transition to standby state, the same runnable in the old context can't
+   * transition RM to standby state again. A new runnable is created every time
+   * RM transitions to active state.
+   */
+  private class StandByTransitionRunnable implements Runnable {
+    // The atomic variable to make sure multiple threads with the same runnable
+    // run only once.
+    private AtomicBoolean hasAlreadyRun = new AtomicBoolean(false);
+
+    @Override
+    public void run() {
+      // Run this only once, even if multiple threads end up triggering
+      // this simultaneously.
+      if (hasAlreadyRun.getAndSet(true)) {
+        return;
+      }
+
+      if (rmContext.isHAEnabled()) {
+        try {
+          // Transition to standby and reinit active services
+          LOG.info("Transitioning RM to Standby mode");
+          transitionToStandby(true);
+          EmbeddedElector elector = rmContext.getLeaderElectorService();
+          if (elector != null) {
+            elector.rejoinElection();
+          }
+        } catch (Exception e) {
+          LOG.fatal("Failed to transition RM to Standby mode.", e);
+          ExitUtil.terminate(1, e);
         }
-      } catch (Exception e) {
-        LOG.fatal("Failed to transition RM to Standby mode.", e);
-        ExitUtil.terminate(1, e);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/652679aa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
index 6ede3b4..5e3cf22 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
@@ -1132,10 +1132,7 @@ public abstract class RMStateStore extends AbstractService {
     if (HAUtil.isHAEnabled(getConfig())) {
       LOG.warn("State-store fenced ! Transitioning RM to standby");
       isFenced = true;
-      Thread standByTransitionThread =
-          new Thread(new StandByTransitionThread());
-      standByTransitionThread.setName("StandByTransitionThread Handler");
-      standByTransitionThread.start();
+      resourceManager.handleTransitionToStandByInNewThread();
     } else if (YarnConfiguration.shouldRMFailFast(getConfig())) {
       LOG.fatal("Fail RM now due to state-store error!");
       rmDispatcher.getEventHandler().handle(
@@ -1200,14 +1197,6 @@ public abstract class RMStateStore extends AbstractService {
     this.resourceManager = rm;
   }
 
-  private class StandByTransitionThread implements Runnable {
-    @Override
-    public void run() {
-      LOG.info("RMStateStore has been fenced");
-      resourceManager.handleTransitionToStandBy();
-    }
-  }
-
   public RMStateStoreState getRMStateStoreState() {
     this.readLock.lock();
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/652679aa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
index ad41b11..7a29735 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
@@ -90,7 +90,7 @@ public class DominantResourceFairnessPolicy extends SchedulingPolicy {
 
   @Override
   public boolean checkIfUsageOverFairShare(Resource usage, Resource fairShare) {
-    return !Resources.fitsIn(usage, fairShare);
+    return Resources.greaterThan(CALCULATOR, null, usage, fairShare);
   }
 
   @Override


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


[40/50] [abbrv] hadoop git commit: HADOOP-14077. Add ability to access jmx via proxy. Contributed by Yuanbo Liu.

Posted by xg...@apache.org.
HADOOP-14077. Add ability to access jmx via proxy.  Contributed by Yuanbo Liu.


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

Branch: refs/heads/YARN-5734
Commit: 172b23af33554b7d58fd41b022d983bcc2433da7
Parents: 3a2e30f
Author: Eric Yang <ey...@apache.org>
Authored: Sat Feb 18 18:34:13 2017 -0800
Committer: Eric Yang <ey...@apache.org>
Committed: Sat Feb 18 18:34:13 2017 -0800

----------------------------------------------------------------------
 .../AuthenticationWithProxyUserFilter.java      |  43 +++---
 .../hadoop/http/TestHttpServerWithSpengo.java   |  15 ++-
 .../mapreduce/v2/app/webapp/AppController.java  |   7 +-
 .../hadoop/yarn/server/webapp/AppBlock.java     | 135 ++++++++++---------
 4 files changed, 114 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/172b23af/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationWithProxyUserFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationWithProxyUserFilter.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationWithProxyUserFilter.java
index ea9b282..751cf02 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationWithProxyUserFilter.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationWithProxyUserFilter.java
@@ -17,10 +17,11 @@
  */
 package org.apache.hadoop.security;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.apache.hadoop.util.HttpExceptionUtils;
 import org.apache.http.NameValuePair;
 import org.apache.http.client.utils.URLEncodedUtils;
 
@@ -41,6 +42,9 @@ import java.util.List;
  */
 public class AuthenticationWithProxyUserFilter extends AuthenticationFilter {
 
+  public static final Log LOG =
+      LogFactory.getLog(AuthenticationWithProxyUserFilter.class);
+
   /**
    * Constant used in URL's query string to perform a proxy user request, the
    * value of the <code>DO_AS</code> parameter is the user the request will be
@@ -66,29 +70,30 @@ public class AuthenticationWithProxyUserFilter extends AuthenticationFilter {
   protected void doFilter(FilterChain filterChain, HttpServletRequest request,
       HttpServletResponse response) throws IOException, ServletException {
 
-    // authorize proxy user before calling next filter.
-    String proxyUser = getDoAs(request);
+    final String proxyUser = getDoAs(request);
     if (proxyUser != null) {
-      UserGroupInformation realUser =
-          UserGroupInformation.createRemoteUser(request.getRemoteUser());
-      UserGroupInformation proxyUserInfo =
-          UserGroupInformation.createProxyUser(proxyUser, realUser);
-
-      try {
-        ProxyUsers.authorize(proxyUserInfo, request.getRemoteAddr());
-      } catch (AuthorizationException ex) {
-        HttpExceptionUtils.createServletExceptionResponse(response,
-            HttpServletResponse.SC_FORBIDDEN, ex);
-        // stop filter chain if there is an Authorization Exception.
-        return;
-      }
 
-      final UserGroupInformation finalProxyUser = proxyUserInfo;
       // Change the remote user after proxy user is authorized.
-      request = new HttpServletRequestWrapper(request) {
+      final HttpServletRequest finalReq = request;
+      request = new HttpServletRequestWrapper(finalReq) {
+
+        private String getRemoteOrProxyUser() throws AuthorizationException {
+          UserGroupInformation realUser =
+              UserGroupInformation.createRemoteUser(finalReq.getRemoteUser());
+          UserGroupInformation proxyUserInfo =
+              UserGroupInformation.createProxyUser(proxyUser, realUser);
+          ProxyUsers.authorize(proxyUserInfo, finalReq.getRemoteAddr());
+          return proxyUserInfo.getUserName();
+        }
+
         @Override
         public String getRemoteUser() {
-          return finalProxyUser.getUserName();
+          try {
+            return getRemoteOrProxyUser();
+          } catch (AuthorizationException ex) {
+            LOG.error("Unable to verify proxy user: " + ex.getMessage(), ex);
+          }
+          return null;
         }
       };
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/172b23af/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWithSpengo.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWithSpengo.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWithSpengo.java
index cbdda90..3d3e020 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWithSpengo.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWithSpengo.java
@@ -157,12 +157,25 @@ public class TestHttpServerWithSpengo {
         Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
       }
 
-      // userA cannot impersonate userC, it fails.
+      // userA cannot impersonate userC, but for /stacks, /jmx and /conf,
+      // they doesn't require users to authorize by default, so they
+      // can be accessed.
       for (String servlet :
           new String[]{"stacks", "jmx", "conf"}){
         HttpURLConnection conn = authUrl
             .openConnection(new URL(serverURL + servlet + "?doAs=userC"),
                 token);
+        Assert.assertEquals(HttpURLConnection.HTTP_OK,
+            conn.getResponseCode());
+      }
+
+      // "/logs" and "/logLevel" require admin authorization,
+      // only userA has the access.
+      for (String servlet :
+          new String[]{"logLevel", "logs"}) {
+        HttpURLConnection conn = authUrl
+            .openConnection(new URL(serverURL + servlet + "?doAs=userC"),
+                token);
         Assert.assertEquals(HttpURLConnection.HTTP_FORBIDDEN,
             conn.getResponseCode());
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/172b23af/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
index e30e1b9..1c90cb9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
@@ -392,10 +392,11 @@ public class AppController extends Controller implements AMParams {
    */
   boolean checkAccess(Job job) {
     String remoteUser = request().getRemoteUser();
-    UserGroupInformation callerUGI = null;
-    if (remoteUser != null) {
-      callerUGI = UserGroupInformation.createRemoteUser(remoteUser);
+    if (remoteUser == null) {
+      return false;
     }
+    UserGroupInformation callerUGI =
+        UserGroupInformation.createRemoteUser(remoteUser);
     if (callerUGI != null && !job.checkAccess(callerUGI, JobACL.VIEW_JOB)) {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/172b23af/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 349a98c..11bd9b4 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
@@ -31,6 +31,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.http.RestCsrfPreventionFilter;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
@@ -100,8 +101,8 @@ public class AppBlock extends HtmlBlock {
       final GetApplicationReportRequest request =
           GetApplicationReportRequest.newInstance(appID);
       if (callerUGI == null) {
-        appReport =
-            appBaseProt.getApplicationReport(request).getApplicationReport();
+        throw new AuthenticationException(
+            "Failed to get user name from request");
       } else {
         appReport = callerUGI.doAs(
             new PrivilegedExceptionAction<ApplicationReport> () {
@@ -165,36 +166,76 @@ public class AppBlock extends HtmlBlock {
     String schedulerPath = WebAppUtils.getResolvedRMWebAppURLWithScheme(conf) +
         "/cluster/scheduler?openQueues=" + app.getQueue();
 
+    generateOverviewTable(app, schedulerPath, webUiType, appReport);
+
+    Collection<ApplicationAttemptReport> attempts;
+    try {
+      final GetApplicationAttemptsRequest request =
+          GetApplicationAttemptsRequest.newInstance(appID);
+      attempts = callerUGI.doAs(
+          new PrivilegedExceptionAction<Collection<
+              ApplicationAttemptReport>>() {
+            @Override
+            public Collection<ApplicationAttemptReport> run() throws Exception {
+              return appBaseProt.getApplicationAttempts(request)
+                  .getApplicationAttemptList();
+            }
+          });
+    } catch (Exception e) {
+      String message =
+          "Failed to read the attempts of the application " + appID + ".";
+      LOG.error(message, e);
+      html.p()._(message)._();
+      return;
+    }
+
+    createApplicationMetricsTable(html);
+
+    html._(InfoBlock.class);
+
+    generateApplicationTable(html, callerUGI, attempts);
+
+  }
+
+  /**
+   * Generate overview table for app web page.
+   * @param app app info.
+   * @param schedulerPath schedule path.
+   * @param webUiType web ui type.
+   * @param appReport app report.
+   */
+  private void generateOverviewTable(AppInfo app, String schedulerPath,
+      String webUiType, ApplicationReport appReport) {
     ResponseInfo overviewTable = info("Application Overview")
-      ._("User:", schedulerPath, app.getUser())
-      ._("Name:", app.getName())
-      ._("Application Type:", app.getType())
-      ._("Application Tags:",
-        app.getApplicationTags() == null ? "" : app.getApplicationTags())
-      ._("Application Priority:", clarifyAppPriority(app.getPriority()))
-      ._(
-        "YarnApplicationState:",
-        app.getAppState() == null ? UNAVAILABLE : clarifyAppState(app
-          .getAppState()))
-      ._("Queue:", schedulerPath, app.getQueue())
-      ._("FinalStatus Reported by AM:",
-        clairfyAppFinalStatus(app.getFinalAppStatus()))
-      ._("Started:", Times.format(app.getStartedTime()))
-      ._(
-        "Elapsed:",
-        StringUtils.formatTime(Times.elapsed(app.getStartedTime(),
-          app.getFinishedTime())))
-      ._(
-        "Tracking URL:",
-        app.getTrackingUrl() == null
-            || app.getTrackingUrl().equals(UNAVAILABLE) ? null : root_url(app
-          .getTrackingUrl()),
-        app.getTrackingUrl() == null
-            || app.getTrackingUrl().equals(UNAVAILABLE) ? "Unassigned" : app
-          .getAppState() == YarnApplicationState.FINISHED
-            || app.getAppState() == YarnApplicationState.FAILED
-            || app.getAppState() == YarnApplicationState.KILLED ? "History"
-            : "ApplicationMaster");
+        ._("User:", schedulerPath, app.getUser())
+        ._("Name:", app.getName())
+        ._("Application Type:", app.getType())
+        ._("Application Tags:",
+            app.getApplicationTags() == null ? "" : app.getApplicationTags())
+        ._("Application Priority:", clarifyAppPriority(app.getPriority()))
+        ._(
+            "YarnApplicationState:",
+            app.getAppState() == null ? UNAVAILABLE : clarifyAppState(app
+                .getAppState()))
+        ._("Queue:", schedulerPath, app.getQueue())
+        ._("FinalStatus Reported by AM:",
+            clairfyAppFinalStatus(app.getFinalAppStatus()))
+        ._("Started:", Times.format(app.getStartedTime()))
+        ._(
+            "Elapsed:",
+            StringUtils.formatTime(Times.elapsed(app.getStartedTime(),
+                app.getFinishedTime())))
+        ._(
+            "Tracking URL:",
+            app.getTrackingUrl() == null
+                || app.getTrackingUrl().equals(UNAVAILABLE) ? null : root_url(app
+                .getTrackingUrl()),
+            app.getTrackingUrl() == null
+                || app.getTrackingUrl().equals(UNAVAILABLE) ? "Unassigned" : app
+                .getAppState() == YarnApplicationState.FINISHED
+                || app.getAppState() == YarnApplicationState.FAILED
+                || app.getAppState() == YarnApplicationState.KILLED ? "History"
+                : "ApplicationMaster");
     if (webUiType != null
         && webUiType.equals(YarnWebParams.RM_WEB_UI)) {
       LogAggregationStatus status = getLogAggregationStatus();
@@ -226,38 +267,6 @@ public class AppBlock extends HtmlBlock {
     overviewTable._("AM container Node Label expression:",
         app.getAmNodeLabelExpression() == null ? "<Not set>"
             : app.getAmNodeLabelExpression());
-
-    Collection<ApplicationAttemptReport> attempts;
-    try {
-      final GetApplicationAttemptsRequest request =
-          GetApplicationAttemptsRequest.newInstance(appID);
-      if (callerUGI == null) {
-        attempts = appBaseProt.getApplicationAttempts(request)
-            .getApplicationAttemptList();
-      } else {
-        attempts = callerUGI.doAs(
-            new PrivilegedExceptionAction<Collection<ApplicationAttemptReport>> () {
-          @Override
-          public Collection<ApplicationAttemptReport> run() throws Exception {
-            return appBaseProt.getApplicationAttempts(request)
-                .getApplicationAttemptList();
-          }
-        });
-      }
-    } catch (Exception e) {
-      String message =
-          "Failed to read the attempts of the application " + appID + ".";
-      LOG.error(message, e);
-      html.p()._(message)._();
-      return;
-    }
-
-    createApplicationMetricsTable(html);
-
-    html._(InfoBlock.class);
-
-    generateApplicationTable(html, callerUGI, attempts);
-
   }
 
   protected void generateApplicationTable(Block html,


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


[16/50] [abbrv] hadoop git commit: YARN-6174. Log files pattern should be same for both running and finished container. Contributed by Xuan Gong.

Posted by xg...@apache.org.
YARN-6174. Log files pattern should be same for both running and finished container. 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/ce2d5bfa
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ce2d5bfa
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ce2d5bfa

Branch: refs/heads/YARN-5734
Commit: ce2d5bfa5f84e7e563980796549b56ef1e4bbf1e
Parents: 859bd15
Author: Junping Du <ju...@apache.org>
Authored: Wed Feb 15 09:05:14 2017 -0800
Committer: Junping Du <ju...@apache.org>
Committed: Wed Feb 15 09:05:14 2017 -0800

----------------------------------------------------------------------
 .../ContainerLogAggregationType.java            | 31 ++++++++++++++++++++
 .../yarn/logaggregation/LogToolUtils.java       | 11 +++----
 .../webapp/AHSWebServices.java                  | 13 ++++----
 .../webapp/TestAHSWebServices.java              | 26 +++++++++-------
 .../server/webapp/dao/ContainerLogsInfo.java    |  8 ++---
 .../nodemanager/webapp/NMWebServices.java       | 11 +++----
 .../webapp/dao/NMContainerLogsInfo.java         |  4 +--
 .../nodemanager/webapp/TestNMWebServices.java   | 19 ++++++------
 8 files changed, 82 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce2d5bfa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogAggregationType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogAggregationType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogAggregationType.java
new file mode 100644
index 0000000..664448b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/ContainerLogAggregationType.java
@@ -0,0 +1,31 @@
+/**
+ * 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.logaggregation;
+
+/**
+ * Enumeration of various aggregation type of a container log.
+ */
+public enum ContainerLogAggregationType {
+
+  /** The log is from NodeManager local log directory. */
+  LOCAL,
+
+  /** The log is from Remote FileSystem application log directory. */
+  AGGREGATED
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce2d5bfa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogToolUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogToolUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogToolUtils.java
index d83a8ae..ae2517a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogToolUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/LogToolUtils.java
@@ -141,7 +141,7 @@ public final class LogToolUtils {
   public static void outputContainerLog(String containerId, String nodeId,
       String fileName, long fileLength, long outputSize,
       String lastModifiedTime, InputStream fis, OutputStream os,
-      byte[] buf, ContainerLogType logType) throws IOException {
+      byte[] buf, ContainerLogAggregationType logType) throws IOException {
     long toSkip = 0;
     long totalBytesToRead = fileLength;
     long skipAfterRead = 0;
@@ -171,9 +171,9 @@ public final class LogToolUtils {
           LogToolUtils.CONTAINER_ON_NODE_PATTERN,
           containerId, nodeId);
       sb.append(containerStr + "\n");
-      sb.append("LogType: " + logType + "\n");
+      sb.append("LogAggregationType: " + logType + "\n");
       sb.append(StringUtils.repeat("=", containerStr.length()) + "\n");
-      sb.append("FileName:" + fileName + "\n");
+      sb.append("LogType:" + fileName + "\n");
       sb.append("LogLastModifiedTime:" + lastModifiedTime + "\n");
       sb.append("LogLength:" + Long.toString(fileLength) + "\n");
       sb.append("LogContents:\n");
@@ -240,9 +240,10 @@ public final class LogToolUtils {
                 LogToolUtils.outputContainerLog(containerId,
                     nodeId, fileType, fileLength, outputSize,
                     Times.format(thisNodeFile.getModificationTime()),
-                    valueStream, os, buf, ContainerLogType.AGGREGATED);
+                    valueStream, os, buf,
+                    ContainerLogAggregationType.AGGREGATED);
                 StringBuilder sb = new StringBuilder();
-                String endOfFile = "End of LogFile:" + fileType;
+                String endOfFile = "End of LogType:" + fileType;
                 sb.append("\n" + endOfFile + "\n");
                 sb.append(StringUtils.repeat("*", endOfFile.length() + 50)
                     + "\n\n");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce2d5bfa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
index 33307a9..c296aaa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
@@ -55,7 +55,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogMeta;
-import org.apache.hadoop.yarn.logaggregation.ContainerLogType;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
 import org.apache.hadoop.yarn.logaggregation.LogToolUtils;
 import org.apache.hadoop.yarn.server.webapp.WebServices;
 import org.apache.hadoop.yarn.server.webapp.YarnWebServiceParams;
@@ -510,13 +510,14 @@ public class AHSWebServices extends WebServices {
         boolean findLogs = LogToolUtils.outputAggregatedContainerLog(conf,
             appId, appOwner, containerIdStr, nodeId, logFile, bytes, os, buf);
         if (!findLogs) {
-          throw new IOException("Can not find logs for container:"
-              + containerIdStr);
+          os.write(("Can not find logs for container:"
+              + containerIdStr).getBytes(Charset.forName("UTF-8")));
         } else {
           if (printEmptyLocalContainerLog) {
             StringBuilder sb = new StringBuilder();
             sb.append(containerIdStr + "\n");
-            sb.append("LogType: " + ContainerLogType.LOCAL + "\n");
+            sb.append("LogAggregationType: "
+                + ContainerLogAggregationType.LOCAL + "\n");
             sb.append("LogContents:\n");
             sb.append(getNoRedirectWarning() + "\n");
             os.write(sb.toString().getBytes(Charset.forName("UTF-8")));
@@ -548,14 +549,14 @@ public class AHSWebServices extends WebServices {
       List<ContainerLogsInfo> containersLogsInfo = new ArrayList<>();
       for (ContainerLogMeta meta : containerLogMeta) {
         ContainerLogsInfo logInfo = new ContainerLogsInfo(meta,
-            ContainerLogType.AGGREGATED);
+            ContainerLogAggregationType.AGGREGATED);
         containersLogsInfo.add(logInfo);
       }
       if (emptyLocalContainerLogMeta) {
         ContainerLogMeta emptyMeta = new ContainerLogMeta(
             containerIdStr, "N/A");
         ContainerLogsInfo empty = new ContainerLogsInfo(emptyMeta,
-            ContainerLogType.LOCAL);
+            ContainerLogAggregationType.LOCAL);
         containersLogsInfo.add(empty);
       }
       GenericEntity<List<ContainerLogsInfo>> meta = new GenericEntity<List<

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce2d5bfa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
index 85bf1b4..c2cfb3b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
@@ -52,7 +52,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.logaggregation.ContainerLogType;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
 import org.apache.hadoop.yarn.logaggregation.PerContainerLogFileInfo;
 import org.apache.hadoop.yarn.logaggregation.TestContainerLogsUtils;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryClientService;
@@ -599,8 +599,8 @@ public class TestAHSWebServices extends JerseyTestBase {
     assertTrue(responseText.contains("Hello." + containerId1ForApp100));
     int fullTextSize = responseText.getBytes().length;
     String tailEndSeparator = StringUtils.repeat("*",
-        "End of LogFile:syslog".length() + 50) + "\n\n";
-    int tailTextSize = "\nEnd of LogFile:syslog\n".getBytes().length
+        "End of LogType:syslog".length() + 50) + "\n\n";
+    int tailTextSize = "\nEnd of LogType:syslog\n".getBytes().length
         + tailEndSeparator.getBytes().length;
 
     String logMessage = "Hello." + containerId1ForApp100;
@@ -752,7 +752,8 @@ public class TestAHSWebServices extends JerseyTestBase {
     assertTrue(responseText.contains(content));
     // Also test whether we output the empty local container log, and give
     // the warning message.
-    assertTrue(responseText.contains("LogType: " + ContainerLogType.LOCAL));
+    assertTrue(responseText.contains("LogAggregationType: "
+        + ContainerLogAggregationType.LOCAL));
     assertTrue(responseText.contains(AHSWebServices.getNoRedirectWarning()));
 
     // If we can not container information from ATS, and we specify the NM id,
@@ -767,7 +768,8 @@ public class TestAHSWebServices extends JerseyTestBase {
         .get(ClientResponse.class);
     responseText = response.getEntity(String.class);
     assertTrue(responseText.contains(content));
-    assertTrue(responseText.contains("LogType: " + ContainerLogType.LOCAL));
+    assertTrue(responseText.contains("LogAggregationType: "
+        + ContainerLogAggregationType.LOCAL));
     assertTrue(responseText.contains(AHSWebServices.getNoRedirectWarning()));
   }
 
@@ -830,7 +832,8 @@ public class TestAHSWebServices extends JerseyTestBase {
         List<ContainerLogsInfo>>(){});
     assertTrue(responseText.size() == 2);
     for (ContainerLogsInfo logInfo : responseText) {
-      if(logInfo.getLogType().equals(ContainerLogType.AGGREGATED.toString())) {
+      if(logInfo.getLogType().equals(
+          ContainerLogAggregationType.AGGREGATED.toString())) {
         List<PerContainerLogFileInfo> logMeta = logInfo
             .getContainerLogsInfo();
         assertTrue(logMeta.size() == 1);
@@ -838,7 +841,8 @@ public class TestAHSWebServices extends JerseyTestBase {
         assertEquals(logMeta.get(0).getFileSize(), String.valueOf(
             content.length()));
       } else {
-        assertEquals(logInfo.getLogType(), ContainerLogType.LOCAL.toString());
+        assertEquals(logInfo.getLogType(),
+            ContainerLogAggregationType.LOCAL.toString());
       }
     }
 
@@ -856,7 +860,8 @@ public class TestAHSWebServices extends JerseyTestBase {
         List<ContainerLogsInfo>>(){});
     assertTrue(responseText.size() == 2);
     for (ContainerLogsInfo logInfo : responseText) {
-      if(logInfo.getLogType().equals(ContainerLogType.AGGREGATED.toString())) {
+      if(logInfo.getLogType().equals(
+          ContainerLogAggregationType.AGGREGATED.toString())) {
         List<PerContainerLogFileInfo> logMeta = logInfo
             .getContainerLogsInfo();
         assertTrue(logMeta.size() == 1);
@@ -864,7 +869,8 @@ public class TestAHSWebServices extends JerseyTestBase {
         assertEquals(logMeta.get(0).getFileSize(), String.valueOf(
             content.length()));
       } else {
-        assertEquals(logInfo.getLogType(), ContainerLogType.LOCAL.toString());
+        assertEquals(logInfo.getLogType(),
+            ContainerLogAggregationType.LOCAL.toString());
       }
     }
   }
@@ -893,7 +899,7 @@ public class TestAHSWebServices extends JerseyTestBase {
         List<ContainerLogsInfo>>(){});
     assertTrue(responseText.size() == 1);
     assertEquals(responseText.get(0).getLogType(),
-        ContainerLogType.AGGREGATED.toString());
+        ContainerLogAggregationType.AGGREGATED.toString());
     List<PerContainerLogFileInfo> logMeta = responseText.get(0)
         .getContainerLogsInfo();
     assertTrue(logMeta.size() == 1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce2d5bfa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerLogsInfo.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/ContainerLogsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerLogsInfo.java
index fd3c43b..bc3ab39 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerLogsInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerLogsInfo.java
@@ -26,7 +26,7 @@ import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogMeta;
-import org.apache.hadoop.yarn.logaggregation.ContainerLogType;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
 import org.apache.hadoop.yarn.logaggregation.PerContainerLogFileInfo;
 
 /**
@@ -48,7 +48,7 @@ public class ContainerLogsInfo {
   @XmlElement(name = "containerLogInfo")
   protected List<PerContainerLogFileInfo> containerLogsInfo;
 
-  @XmlElement(name = "logType")
+  @XmlElement(name = "logAggregationType")
   protected String logType;
 
   @XmlElement(name = "containerId")
@@ -60,8 +60,8 @@ public class ContainerLogsInfo {
   //JAXB needs this
   public ContainerLogsInfo() {}
 
-  public ContainerLogsInfo(ContainerLogMeta logMeta, ContainerLogType logType)
-      throws YarnException {
+  public ContainerLogsInfo(ContainerLogMeta logMeta,
+      ContainerLogAggregationType logType) throws YarnException {
     this.containerLogsInfo = new ArrayList<PerContainerLogFileInfo>(
         logMeta.getContainerLogMeta());
     this.logType = logType.toString();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce2d5bfa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
index 44b232d..4e72746 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMWebServices.java
@@ -55,7 +55,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogMeta;
-import org.apache.hadoop.yarn.logaggregation.ContainerLogType;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
 import org.apache.hadoop.yarn.logaggregation.LogToolUtils;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.ResourceView;
@@ -253,7 +253,7 @@ public class NMWebServices {
       List<ContainerLogsInfo> containersLogsInfo = new ArrayList<>();
       containersLogsInfo.add(new NMContainerLogsInfo(
           this.nmContext, containerId,
-          hsr.getRemoteUser(), ContainerLogType.LOCAL));
+          hsr.getRemoteUser(), ContainerLogAggregationType.LOCAL));
       // check whether we have aggregated logs in RemoteFS. If exists, show the
       // the log meta for the aggregated logs as well.
       ApplicationId appId = containerId.getApplicationAttemptId()
@@ -268,7 +268,7 @@ public class NMWebServices {
         if (!containerLogMeta.isEmpty()) {
           for (ContainerLogMeta logMeta : containerLogMeta) {
             containersLogsInfo.add(new ContainerLogsInfo(logMeta,
-                ContainerLogType.AGGREGATED));
+                ContainerLogAggregationType.AGGREGATED));
           }
         }
       } catch (IOException ex) {
@@ -429,9 +429,10 @@ public class NMWebServices {
             byte[] buf = new byte[bufferSize];
             LogToolUtils.outputContainerLog(containerId.toString(),
                 nmContext.getNodeId().toString(), outputFileName, fileLength,
-                bytes, lastModifiedTime, fis, os, buf, ContainerLogType.LOCAL);
+                bytes, lastModifiedTime, fis, os, buf,
+                ContainerLogAggregationType.LOCAL);
             StringBuilder sb = new StringBuilder();
-            String endOfFile = "End of LogFile:" + outputFileName;
+            String endOfFile = "End of LogType:" + outputFileName;
             sb.append(endOfFile + ".");
             if (isRunning) {
               sb.append("This log file belongs to a running container ("

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce2d5bfa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/NMContainerLogsInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/NMContainerLogsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/NMContainerLogsInfo.java
index d2bc6f2..5415e04 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/NMContainerLogsInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/NMContainerLogsInfo.java
@@ -26,7 +26,7 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.logaggregation.ContainerLogType;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
 import org.apache.hadoop.yarn.logaggregation.PerContainerLogFileInfo;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.webapp.ContainerLogsUtils;
@@ -47,7 +47,7 @@ public class NMContainerLogsInfo extends ContainerLogsInfo {
 
   public NMContainerLogsInfo(final Context nmContext,
       final ContainerId containerId, String remoteUser,
-      ContainerLogType logType) throws YarnException {
+      ContainerLogAggregationType logType) throws YarnException {
     this.logType = logType.toString();
     this.containerId = containerId.toString();
     this.nodeId = nmContext.getNodeId().toString();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce2d5bfa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
index e3773d9..d9fd289 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
@@ -50,7 +50,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.hadoop.yarn.logaggregation.ContainerLogType;
+import org.apache.hadoop.yarn.logaggregation.ContainerLogAggregationType;
 import org.apache.hadoop.yarn.logaggregation.PerContainerLogFileInfo;
 import org.apache.hadoop.yarn.logaggregation.TestContainerLogsUtils;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
@@ -530,7 +530,7 @@ public class TestNMWebServices extends JerseyTestBase {
         List<ContainerLogsInfo>>(){});
     assertTrue(responseList.size() == 1);
     assertEquals(responseList.get(0).getLogType(),
-        ContainerLogType.LOCAL.toString());
+        ContainerLogAggregationType.LOCAL.toString());
     List<PerContainerLogFileInfo> logMeta = responseList.get(0)
         .getContainerLogsInfo();
     assertTrue(logMeta.size() == 1);
@@ -557,12 +557,13 @@ public class TestNMWebServices extends JerseyTestBase {
       assertEquals(responseList.size(), 2);
       for (ContainerLogsInfo logInfo : responseList) {
         if(logInfo.getLogType().equals(
-            ContainerLogType.AGGREGATED.toString())) {
+            ContainerLogAggregationType.AGGREGATED.toString())) {
           List<PerContainerLogFileInfo> meta = logInfo.getContainerLogsInfo();
           assertTrue(meta.size() == 1);
           assertEquals(meta.get(0).getFileName(), aggregatedLogFile);
         } else {
-          assertEquals(logInfo.getLogType(), ContainerLogType.LOCAL.toString());
+          assertEquals(logInfo.getLogType(),
+              ContainerLogAggregationType.LOCAL.toString());
           List<PerContainerLogFileInfo> meta = logInfo.getContainerLogsInfo();
           assertTrue(meta.size() == 1);
           assertEquals(meta.get(0).getFileName(), filename);
@@ -577,11 +578,11 @@ public class TestNMWebServices extends JerseyTestBase {
       response = r.path(filename)
           .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
       responseText = response.getEntity(String.class);
-      assertTrue(responseText.contains("LogType: "
-          + ContainerLogType.AGGREGATED));
+      assertTrue(responseText.contains("LogAggregationType: "
+          + ContainerLogAggregationType.AGGREGATED));
       assertTrue(responseText.contains(aggregatedLogMessage));
-      assertTrue(responseText.contains("LogType: "
-              + ContainerLogType.LOCAL));
+      assertTrue(responseText.contains("LogAggregationType: "
+              + ContainerLogAggregationType.LOCAL));
       assertTrue(responseText.contains(logMessage));
     } finally {
       FileUtil.fullyDelete(tempLogDir);
@@ -685,7 +686,7 @@ public class TestNMWebServices extends JerseyTestBase {
 
   private String getLogContext(String fullMessage) {
     String prefix = "LogContents:\n";
-    String postfix = "End of LogFile:";
+    String postfix = "End of LogType:";
     int prefixIndex = fullMessage.indexOf(prefix) + prefix.length();
     int postfixIndex = fullMessage.indexOf(postfix);
     return fullMessage.substring(prefixIndex, postfixIndex);


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


[22/50] [abbrv] hadoop git commit: YARN-4212. FairScheduler: Can't create a DRF queue under a FAIR policy queue. (Yufei Gu via kasha)

Posted by xg...@apache.org.
YARN-4212. FairScheduler: Can't create a DRF queue under a FAIR policy queue. (Yufei Gu via kasha)


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

Branch: refs/heads/YARN-5734
Commit: 11be3f70e029c2324b167563168c8a254d234aef
Parents: 6c25dbc
Author: Karthik Kambatla <ka...@cloudera.com>
Authored: Wed Feb 15 23:51:22 2017 -0800
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Wed Feb 15 23:51:22 2017 -0800

----------------------------------------------------------------------
 .../scheduler/fair/AllocationConfiguration.java |  11 +-
 .../scheduler/fair/FSLeafQueue.java             |   9 -
 .../scheduler/fair/FSParentQueue.java           |  13 -
 .../resourcemanager/scheduler/fair/FSQueue.java |  50 ++-
 .../scheduler/fair/QueueManager.java            |  28 +-
 .../scheduler/fair/SchedulingPolicy.java        |  36 +--
 .../DominantResourceFairnessPolicy.java         |   5 -
 .../fair/policies/FairSharePolicy.java          |  15 +-
 .../scheduler/fair/policies/FifoPolicy.java     |  14 +-
 .../scheduler/fair/TestFSAppStarvation.java     |  10 +-
 .../scheduler/fair/TestFairScheduler.java       |   1 -
 .../scheduler/fair/TestSchedulingPolicy.java    | 302 +++++++++++++++----
 12 files changed, 340 insertions(+), 154 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/11be3f70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
index 7bd2616..f143aa6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
@@ -408,9 +408,8 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
    * Initialize a {@link FSQueue} with queue-specific properties and its
    * metrics.
    * @param queue the FSQueue needed to be initialized
-   * @param scheduler the scheduler which the queue belonged to
    */
-  public void initFSQueue(FSQueue queue, FairScheduler scheduler){
+  public void initFSQueue(FSQueue queue){
     // Set queue-specific properties.
     String name = queue.getName();
     queue.setWeights(getQueueWeight(name));
@@ -419,14 +418,6 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
     queue.setMaxRunningApps(getQueueMaxApps(name));
     queue.setMaxAMShare(getQueueMaxAMShare(name));
     queue.setMaxChildQueueResource(getMaxChildResources(name));
-    try {
-      SchedulingPolicy policy = getSchedulingPolicy(name);
-      policy.initialize(scheduler.getClusterResource());
-      queue.setPolicy(policy);
-    } catch (AllocationConfigurationException ex) {
-      LOG.warn("Failed to set the scheduling policy "
-          + getDefaultSchedulingPolicy(), ex);
-    }
 
     // Set queue metrics.
     queue.getMetrics().setMinShare(getMinResources(name));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11be3f70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
index c4b2de6..59bde5b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
@@ -198,15 +198,6 @@ public class FSLeafQueue extends FSQueue {
   }
 
   @Override
-  public void setPolicy(SchedulingPolicy policy)
-      throws AllocationConfigurationException {
-    if (!SchedulingPolicy.isApplicableTo(policy, SchedulingPolicy.DEPTH_LEAF)) {
-      throwPolicyDoesnotApplyException(policy);
-    }
-    super.policy = policy;
-  }
-
-  @Override
   public void updateInternal(boolean checkStarvation) {
     readLock.lock();
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11be3f70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
index 2528f3d..1c8e9ce 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
@@ -239,19 +239,6 @@ public class FSParentQueue extends FSQueue {
     }
   }
 
-  @Override
-  public void setPolicy(SchedulingPolicy policy)
-      throws AllocationConfigurationException {
-    boolean allowed =
-        SchedulingPolicy.isApplicableTo(policy, (parent == null)
-            ? SchedulingPolicy.DEPTH_ROOT
-            : SchedulingPolicy.DEPTH_INTERMEDIATE);
-    if (!allowed) {
-      throwPolicyDoesnotApplyException(policy);
-    }
-    super.policy = policy;
-  }
-
   void incrementRunnableApps() {
     writeLock.lock();
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11be3f70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
index ee4c35a..7e8b858 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
@@ -91,20 +91,23 @@ public abstract class FSQueue implements Queue, Schedulable {
     this.queueEntity = new PrivilegedEntity(EntityType.QUEUE, name);
     this.metrics = FSQueueMetrics.forQueue(getName(), parent, true, scheduler.getConf());
     this.parent = parent;
+    setPolicy(scheduler.getAllocationConfiguration().getSchedulingPolicy(name));
     reinit(false);
   }
 
   /**
    * Initialize a queue by setting its queue-specific properties and its
-   * metrics.
-   * This function is invoked when a new queue is created or reloading the
-   * allocation configuration.
+   * metrics. This method is invoked when creating a new queue or reloading
+   * the allocation file.
+   * This method does not set policies for queues when reloading the allocation
+   * file since we need to either set all new policies or nothing, which is
+   * handled by method {@link #verifyAndSetPolicyFromConf}.
    *
    * @param recursive whether child queues should be reinitialized recursively
    */
   public void reinit(boolean recursive) {
     AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
-    allocConf.initFSQueue(this, scheduler);
+    allocConf.initFSQueue(this);
     updatePreemptionVariables();
 
     if (recursive) {
@@ -131,15 +134,11 @@ public abstract class FSQueue implements Queue, Schedulable {
     return parent;
   }
 
-  protected void throwPolicyDoesnotApplyException(SchedulingPolicy policy)
-      throws AllocationConfigurationException {
-    throw new AllocationConfigurationException("SchedulingPolicy " + policy
-        + " does not apply to queue " + getName());
+  public void setPolicy(SchedulingPolicy policy) {
+    policy.initialize(scheduler.getClusterResource());
+    this.policy = policy;
   }
 
-  public abstract void setPolicy(SchedulingPolicy policy)
-      throws AllocationConfigurationException;
-
   public void setWeights(ResourceWeights weights){
     this.weights = weights;
   }
@@ -463,4 +462,33 @@ public abstract class FSQueue implements Queue, Schedulable {
     }
     return true;
   }
+
+  /**
+   * Recursively check policies for queues in pre-order. Get queue policies
+   * from the allocation file instead of properties of {@link FSQueue} objects.
+   * Set the policy for current queue if there is no policy violation for its
+   * children. This method is invoked while reloading the allocation file.
+   *
+   * @param queueConf allocation configuration
+   * @return true if no policy violation and successfully set polices
+   *         for queues; false otherwise
+   */
+  public boolean verifyAndSetPolicyFromConf(AllocationConfiguration queueConf) {
+    SchedulingPolicy queuePolicy = queueConf.getSchedulingPolicy(getName());
+
+    for (FSQueue child : getChildQueues()) {
+      if (!queuePolicy.isChildPolicyAllowed(
+          queueConf.getSchedulingPolicy(child.getName()))) {
+        return false;
+      }
+      boolean success = child.verifyAndSetPolicyFromConf(queueConf);
+      if (!success) {
+        return false;
+      }
+    }
+
+    // Set the policy if no policy violation for all children
+    setPolicy(queuePolicy);
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11be3f70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
index 934bcfd..3c601fa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
 import org.xml.sax.SAXException;
 
 import com.google.common.base.CharMatcher;
@@ -42,10 +43,10 @@ import com.google.common.annotations.VisibleForTesting;
 import java.util.Iterator;
 import java.util.Set;
 import org.apache.hadoop.yarn.api.records.Resource;
+
 /**
  * Maintains a list of queues as well as scheduling parameters for each queue,
  * such as guaranteed share allocations, from the fair scheduler config file.
- * 
  */
 @Private
 @Unstable
@@ -72,6 +73,9 @@ public class QueueManager {
 
   public void initialize(Configuration conf) throws IOException,
       SAXException, AllocationConfigurationException, ParserConfigurationException {
+    // Policies of root and default queue are set to
+    // SchedulingPolicy.DEFAULT_POLICY since the allocation file hasn't been
+    // loaded yet.
     rootQueue = new FSParentQueue("root", scheduler, null);
     queues.put(rootQueue.getName(), rootQueue);
 
@@ -80,7 +84,7 @@ public class QueueManager {
     // Recursively reinitialize to propagate queue properties
     rootQueue.reinit(true);
   }
-  
+
   /**
    * Get a leaf queue by name, creating it if the create param is true and is necessary.
    * If the queue is not or can not be a leaf queue, i.e. it already exists as a
@@ -272,12 +276,25 @@ public class QueueManager {
       FSParentQueue newParent = null;
       String queueName = i.next();
 
+      // Check if child policy is allowed
+      SchedulingPolicy childPolicy = scheduler.getAllocationConfiguration().
+          getSchedulingPolicy(queueName);
+      if (!parent.getPolicy().isChildPolicyAllowed(childPolicy)) {
+        LOG.error("Can't create queue '" + queueName + "'.");
+        return null;
+      }
+
       // Only create a leaf queue at the very end
       if (!i.hasNext() && (queueType != FSQueueType.PARENT)) {
         FSLeafQueue leafQueue = new FSLeafQueue(queueName, scheduler, parent);
         leafQueues.add(leafQueue);
         queue = leafQueue;
       } else {
+        if (childPolicy instanceof FifoPolicy) {
+          LOG.error("Can't create queue '" + queueName + "', since "
+              + FifoPolicy.NAME + " is only for leaf queues.");
+          return null;
+        }
         newParent = new FSParentQueue(queueName, scheduler, parent);
         queue = newParent;
       }
@@ -479,6 +496,13 @@ public class QueueManager {
   public void updateAllocationConfiguration(AllocationConfiguration queueConf) {
     // Create leaf queues and the parent queues in a leaf's ancestry if they do not exist
     synchronized (queues) {
+      // Verify and set scheduling policies for existing queues before creating
+      // any queue, since we need parent policies to determine if we can create
+      // its children.
+      if (!rootQueue.verifyAndSetPolicyFromConf(queueConf)) {
+        LOG.error("Setting scheduling policies for existing queues failed!");
+      }
+
       for (String name : queueConf.getConfiguredQueues().get(
               FSQueueType.LEAF)) {
         if (removeEmptyIncompatibleQueues(name, FSQueueType.LEAF)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11be3f70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
index 9eda46c..3fe36f3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
@@ -42,12 +42,6 @@ public abstract class SchedulingPolicy {
   public static final SchedulingPolicy DEFAULT_POLICY =
       getInstance(FairSharePolicy.class);
   
-  public static final byte DEPTH_LEAF = (byte) 1;
-  public static final byte DEPTH_INTERMEDIATE = (byte) 2;
-  public static final byte DEPTH_ROOT = (byte) 4;
-  public static final byte DEPTH_PARENT = (byte) 6; // Root and Intermediate
-  public static final byte DEPTH_ANY = (byte) 7;
-
   /**
    * Returns a {@link SchedulingPolicy} instance corresponding to the passed clazz
    */
@@ -114,27 +108,6 @@ public abstract class SchedulingPolicy {
   public abstract String getName();
 
   /**
-   * Specifies the depths in the hierarchy, this {@link SchedulingPolicy}
-   * applies to
-   * 
-   * @return depth equal to one of fields {@link SchedulingPolicy}#DEPTH_*
-   */
-  public abstract byte getApplicableDepth();
-
-  /**
-   * Checks if the specified {@link SchedulingPolicy} can be used for a queue at
-   * the specified depth in the hierarchy
-   * 
-   * @param policy {@link SchedulingPolicy} we are checking the
-   *          depth-applicability for
-   * @param depth queue's depth in the hierarchy
-   * @return true if policy is applicable to passed depth, false otherwise
-   */
-  public static boolean isApplicableTo(SchedulingPolicy policy, byte depth) {
-    return ((policy.getApplicableDepth() & depth) == depth) ? true : false;
-  }
-
-  /**
    * The comparator returned by this method is to be used for sorting the
    * {@link Schedulable}s in that queue.
    * 
@@ -191,4 +164,13 @@ public abstract class SchedulingPolicy {
   public abstract Resource getHeadroom(Resource queueFairShare,
       Resource queueUsage, Resource maxAvailable);
 
+  /**
+   * Check whether the policy of a child queue is allowed.
+   *
+   * @param childPolicy the policy of child queue
+   * @return true if the child policy is allowed; false otherwise
+   */
+  public boolean isChildPolicyAllowed(SchedulingPolicy childPolicy) {
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11be3f70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
index ad41b11..6f04cb7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
@@ -58,11 +58,6 @@ public class DominantResourceFairnessPolicy extends SchedulingPolicy {
   }
 
   @Override
-  public byte getApplicableDepth() {
-    return SchedulingPolicy.DEPTH_ANY;
-  }
-
-  @Override
   public Comparator<Schedulable> getComparator() {
     return COMPARATOR;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11be3f70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
index d47ea07..9036a03 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
@@ -21,6 +21,8 @@ import java.io.Serializable;
 import java.util.Collection;
 import java.util.Comparator;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -40,6 +42,7 @@ import com.google.common.annotations.VisibleForTesting;
 @Private
 @Unstable
 public class FairSharePolicy extends SchedulingPolicy {
+  private static final Log LOG = LogFactory.getLog(FifoPolicy.class);
   @VisibleForTesting
   public static final String NAME = "fair";
   private static final DefaultResourceCalculator RESOURCE_CALCULATOR =
@@ -175,7 +178,15 @@ public class FairSharePolicy extends SchedulingPolicy {
   }
 
   @Override
-  public byte getApplicableDepth() {
-    return SchedulingPolicy.DEPTH_ANY;
+  public boolean isChildPolicyAllowed(SchedulingPolicy childPolicy) {
+    if (childPolicy instanceof DominantResourceFairnessPolicy) {
+      LOG.error("Queue policy can't be " + DominantResourceFairnessPolicy.NAME
+          + " if the parent policy is " + getName() + ". Choose " +
+          getName() + " or " + FifoPolicy.NAME + " for child queues instead."
+          + " Please note that " + FifoPolicy.NAME
+          + " is only for leaf queues.");
+      return false;
+    }
+    return true;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11be3f70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
index 3e2cb9f..7dd45cb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FifoPolicy.java
@@ -21,14 +21,14 @@ import java.io.Serializable;
 import java.util.Collection;
 import java.util.Comparator;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.Schedulable;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy;
-
-
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -38,6 +38,8 @@ import com.google.common.annotations.VisibleForTesting;
 @Private
 @Unstable
 public class FifoPolicy extends SchedulingPolicy {
+  private static final Log LOG = LogFactory.getLog(FifoPolicy.class);
+
   @VisibleForTesting
   public static final String NAME = "FIFO";
   private static final FifoComparator COMPARATOR = new FifoComparator();
@@ -127,9 +129,11 @@ public class FifoPolicy extends SchedulingPolicy {
     return headroom;
   }
 
-
   @Override
-  public byte getApplicableDepth() {
-    return SchedulingPolicy.DEPTH_LEAF;
+  public boolean isChildPolicyAllowed(SchedulingPolicy childPolicy) {
+    LOG.error(getName() + " policy is only for leaf queues. Please choose "
+        + DominantResourceFairnessPolicy.NAME + " or " + FairSharePolicy.NAME
+        + " for parent queues.");
+    return false;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11be3f70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppStarvation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppStarvation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppStarvation.java
index 3a79ac0..2eacc9e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppStarvation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppStarvation.java
@@ -197,7 +197,7 @@ public class TestFSAppStarvation extends FairSchedulerTestBase {
     out.println("<fairSharePreemptionTimeout>0" +
         "</fairSharePreemptionTimeout>");
     out.println("<schedulingPolicy>fair</schedulingPolicy>");
-    addChildQueue(out);
+    addChildQueue(out, "fair");
     out.println("</queue>");
 
     // DRF queue with fairshare preemption enabled
@@ -207,9 +207,10 @@ public class TestFSAppStarvation extends FairSchedulerTestBase {
     out.println("<fairSharePreemptionTimeout>0" +
         "</fairSharePreemptionTimeout>");
     out.println("<schedulingPolicy>drf</schedulingPolicy>");
-    addChildQueue(out);
+    addChildQueue(out, "drf");
     out.println("</queue>");
-
+    out.println("<defaultQueueSchedulingPolicy>drf" +
+        "</defaultQueueSchedulingPolicy>");
     out.println("</allocations>");
     out.close();
 
@@ -237,13 +238,14 @@ public class TestFSAppStarvation extends FairSchedulerTestBase {
     assertEquals(8, scheduler.getSchedulerApp(app).getLiveContainers().size());
   }
 
-  private void addChildQueue(PrintWriter out) {
+  private void addChildQueue(PrintWriter out, String policy) {
     // Child queue under fairshare with same settings
     out.println("<queue name=\"child\">");
     out.println("<fairSharePreemptionThreshold>1" +
         "</fairSharePreemptionThreshold>");
     out.println("<fairSharePreemptionTimeout>0" +
         "</fairSharePreemptionTimeout>");
+    out.println("<schedulingPolicy>" + policy + "</schedulingPolicy>");
     out.println("</queue>");
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11be3f70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index da5d3ad..0c3a614 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -5164,5 +5164,4 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         Resources.equals(aQueue.getDemand(), maxResource) &&
         Resources.equals(bQueue.getDemand(), maxResource));
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11be3f70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java
index bd49cca..8dccf6e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java
@@ -18,9 +18,10 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.Stack;
@@ -30,16 +31,29 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FairSharePolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
-import org.mockito.Mockito;
+
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNotNull;
 
 public class TestSchedulingPolicy {
   private static final Log LOG = LogFactory.getLog(TestSchedulingPolicy.class);
+  private final static String ALLOC_FILE =
+      new File(FairSchedulerTestBase.TEST_DIR, "test-queues").getAbsolutePath();
+  private FairSchedulerConfiguration conf;
+  private FairScheduler scheduler;
+
+  @Before
+  public void setUp() throws Exception {
+    scheduler = new FairScheduler();
+    conf = new FairSchedulerConfiguration();
+  }
 
   @Test(timeout = 1000)
   public void testParseSchedulingPolicy()
@@ -79,66 +93,6 @@ public class TestSchedulingPolicy {
   }
 
   /**
-   * Trivial tests that make sure
-   * {@link SchedulingPolicy#isApplicableTo(SchedulingPolicy, byte)} works as
-   * expected for the possible values of depth
-   * 
-   * @throws AllocationConfigurationException
-   */
-  @Test(timeout = 1000)
-  public void testIsApplicableTo() throws AllocationConfigurationException {
-    final String ERR = "Broken SchedulingPolicy#isApplicableTo";
-    
-    // fifo
-    SchedulingPolicy policy = SchedulingPolicy.parse("fifo");
-    assertTrue(ERR,
-        SchedulingPolicy.isApplicableTo(policy, SchedulingPolicy.DEPTH_LEAF));
-    assertFalse(ERR, SchedulingPolicy.isApplicableTo(
-        SchedulingPolicy.parse("fifo"), SchedulingPolicy.DEPTH_INTERMEDIATE));
-    assertFalse(ERR, SchedulingPolicy.isApplicableTo(
-        SchedulingPolicy.parse("fifo"), SchedulingPolicy.DEPTH_ROOT));
-
-    
-    // fair
-    policy = SchedulingPolicy.parse("fair"); 
-    assertTrue(ERR,
-        SchedulingPolicy.isApplicableTo(policy, SchedulingPolicy.DEPTH_LEAF));
-    assertTrue(ERR, SchedulingPolicy.isApplicableTo(policy,
-        SchedulingPolicy.DEPTH_INTERMEDIATE));
-    assertTrue(ERR,
-        SchedulingPolicy.isApplicableTo(policy, SchedulingPolicy.DEPTH_ROOT));
-    assertTrue(ERR,
-        SchedulingPolicy.isApplicableTo(policy, SchedulingPolicy.DEPTH_PARENT));
-    assertTrue(ERR,
-        SchedulingPolicy.isApplicableTo(policy, SchedulingPolicy.DEPTH_ANY));
-    
-    // drf
-    policy = SchedulingPolicy.parse("drf"); 
-    assertTrue(ERR,
-        SchedulingPolicy.isApplicableTo(policy, SchedulingPolicy.DEPTH_LEAF));
-    assertTrue(ERR, SchedulingPolicy.isApplicableTo(policy,
-        SchedulingPolicy.DEPTH_INTERMEDIATE));
-    assertTrue(ERR,
-        SchedulingPolicy.isApplicableTo(policy, SchedulingPolicy.DEPTH_ROOT));
-    assertTrue(ERR,
-        SchedulingPolicy.isApplicableTo(policy, SchedulingPolicy.DEPTH_PARENT));
-    assertTrue(ERR,
-        SchedulingPolicy.isApplicableTo(policy, SchedulingPolicy.DEPTH_ANY));
-    
-    policy = Mockito.mock(SchedulingPolicy.class);
-    Mockito.when(policy.getApplicableDepth()).thenReturn(
-        SchedulingPolicy.DEPTH_PARENT);
-    assertTrue(ERR, SchedulingPolicy.isApplicableTo(policy,
-        SchedulingPolicy.DEPTH_INTERMEDIATE));
-    assertTrue(ERR,
-        SchedulingPolicy.isApplicableTo(policy, SchedulingPolicy.DEPTH_ROOT));
-    assertTrue(ERR,
-        SchedulingPolicy.isApplicableTo(policy, SchedulingPolicy.DEPTH_PARENT));
-    assertFalse(ERR,
-        SchedulingPolicy.isApplicableTo(policy, SchedulingPolicy.DEPTH_ANY));
-  }
-
-  /**
    * Test whether {@link FairSharePolicy.FairShareComparator} is transitive.
    */
   @Test
@@ -353,4 +307,222 @@ public class TestSchedulingPolicy {
     }
   }
 
+  @Test
+  public void testSchedulingPolicyViolation() throws IOException {
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"root\">");
+    out.println("<schedulingPolicy>fair</schedulingPolicy>");
+    out.println("    <queue name=\"child1\">");
+    out.println("    <schedulingPolicy>drf</schedulingPolicy>");
+    out.println("    </queue>");
+    out.println("    <queue name=\"child2\">");
+    out.println("    <schedulingPolicy>fair</schedulingPolicy>");
+    out.println("    </queue>");
+    out.println("</queue>");
+    out.println("<defaultQueueSchedulingPolicy>drf" +
+        "</defaultQueueSchedulingPolicy>");
+    out.println("</allocations>");
+    out.close();
+
+    scheduler.init(conf);
+
+    FSQueue child1 = scheduler.getQueueManager().getQueue("child1");
+    assertNull("Queue 'child1' should be null since its policy isn't allowed to"
+        + " be 'drf' if its parent policy is 'fair'.", child1);
+
+    // dynamic queue
+    FSQueue dynamicQueue = scheduler.getQueueManager().
+        getLeafQueue("dynamicQueue", true);
+    assertNull("Dynamic queue should be null since it isn't allowed to be 'drf'"
+        + " policy if its parent policy is 'fair'.", dynamicQueue);
+
+    // Set child1 to 'fair' and child2 to 'drf', the reload the allocation file.
+    out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"root\">");
+    out.println("<schedulingPolicy>fair</schedulingPolicy>");
+    out.println("    <queue name=\"child1\">");
+    out.println("    <schedulingPolicy>fair</schedulingPolicy>");
+    out.println("    </queue>");
+    out.println("    <queue name=\"child2\">");
+    out.println("    <schedulingPolicy>drf</schedulingPolicy>");
+    out.println("    </queue>");
+    out.println("</queue>");
+    out.println("<defaultQueueSchedulingPolicy>drf" +
+        "</defaultQueueSchedulingPolicy>");
+    out.println("</allocations>");
+    out.close();
+
+    scheduler.reinitialize(conf, null);
+    child1 = scheduler.getQueueManager().getQueue("child1");
+    assertNotNull("Queue 'child1' should be not null since its policy is "
+        + "allowed to be 'fair' if its parent policy is 'fair'.", child1);
+
+    // Detect the policy violation of Child2, keep the original policy instead
+    // of setting the new policy.
+    FSQueue child2 = scheduler.getQueueManager().getQueue("child2");
+    assertTrue("Queue 'child2' should be 'fair' since its new policy 'drf' "
+        + "is not allowed.", child2.getPolicy() instanceof FairSharePolicy);
+  }
+
+  @Test
+  public void testSchedulingPolicyViolationInTheMiddleLevel()
+      throws IOException {
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"root\">");
+    out.println("<schedulingPolicy>fair</schedulingPolicy>");
+    out.println("  <queue name=\"level2\">");
+    out.println("    <schedulingPolicy>fair</schedulingPolicy>");
+    out.println("    <queue name=\"level3\">");
+    out.println("    <schedulingPolicy>drf</schedulingPolicy>");
+    out.println("       <queue name=\"leaf\">");
+    out.println("       <schedulingPolicy>fair</schedulingPolicy>");
+    out.println("       </queue>");
+    out.println("    </queue>");
+    out.println("  </queue>");
+    out.println("</queue>");
+    out.println("</allocations>");
+    out.close();
+
+    scheduler.init(conf);
+
+    FSQueue level2 = scheduler.getQueueManager().getQueue("level2");
+    assertNotNull("Queue 'level2' shouldn't be null since its policy is allowed"
+        + " to be 'fair' if its parent policy is 'fair'.", level2);
+    FSQueue level3 = scheduler.getQueueManager().getQueue("level2.level3");
+    assertNull("Queue 'level3' should be null since its policy isn't allowed"
+        + " to be 'drf' if its parent policy is 'fair'.", level3);
+    FSQueue leaf = scheduler.getQueueManager().getQueue("level2.level3.leaf");
+    assertNull("Queue 'leaf' should be null since its parent failed to create.",
+        leaf);
+  }
+
+  @Test
+  public void testFIFOPolicyOnlyForLeafQueues()
+      throws IOException {
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"root\">");
+    out.println("  <queue name=\"intermediate\">");
+    out.println("    <schedulingPolicy>fifo</schedulingPolicy>");
+    out.println("    <queue name=\"leaf\">");
+    out.println("    <schedulingPolicy>fair</schedulingPolicy>");
+    out.println("    </queue>");
+    out.println("  </queue>");
+    out.println("</queue>");
+    out.println("</allocations>");
+    out.close();
+
+    scheduler.init(conf);
+
+    FSQueue intermediate = scheduler.getQueueManager().getQueue("intermediate");
+    assertNull("Queue 'intermediate' should be null since 'fifo' is only for "
+        + "leaf queue.", intermediate);
+
+    out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"root\">");
+    out.println("  <queue name=\"intermediate\">");
+    out.println("    <schedulingPolicy>fair</schedulingPolicy>");
+    out.println("    <queue name=\"leaf\">");
+    out.println("    <schedulingPolicy>fifo</schedulingPolicy>");
+    out.println("    </queue>");
+    out.println("  </queue>");
+    out.println("</queue>");
+    out.println("</allocations>");
+    out.close();
+
+    scheduler.reinitialize(conf, null);
+
+    assertNotNull(scheduler.getQueueManager().getQueue("intermediate"));
+
+    FSQueue leaf = scheduler.getQueueManager().getQueue("intermediate.leaf");
+    assertNotNull("Queue 'leaf' should be null since 'fifo' is only for "
+        + "leaf queue.", leaf);
+  }
+
+  @Test
+  public void testPolicyReinitilization() throws IOException {
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"root\">");
+    out.println("<schedulingPolicy>fair</schedulingPolicy>");
+    out.println("    <queue name=\"child1\">");
+    out.println("    <schedulingPolicy>fair</schedulingPolicy>");
+    out.println("    </queue>");
+    out.println("    <queue name=\"child2\">");
+    out.println("    <schedulingPolicy>fair</schedulingPolicy>");
+    out.println("    </queue>");
+    out.println("</queue>");
+    out.println("</allocations>");
+    out.close();
+
+    scheduler.init(conf);
+
+    // Set child1 to 'drf' which is not allowed, then reload the allocation file
+    out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"root\">");
+    out.println("<schedulingPolicy>fair</schedulingPolicy>");
+    out.println("    <queue name=\"child1\">");
+    out.println("    <schedulingPolicy>drf</schedulingPolicy>");
+    out.println("    </queue>");
+    out.println("    <queue name=\"child2\">");
+    out.println("    <schedulingPolicy>fifo</schedulingPolicy>");
+    out.println("    </queue>");
+    out.println("</queue>");
+    out.println("</allocations>");
+    out.close();
+
+    scheduler.reinitialize(conf, null);
+
+    FSQueue child1 = scheduler.getQueueManager().getQueue("child1");
+    assertTrue("Queue 'child1' should still be 'fair' since 'drf' isn't allowed"
+            + " if its parent policy is 'fair'.",
+        child1.getPolicy() instanceof FairSharePolicy);
+    FSQueue child2 = scheduler.getQueueManager().getQueue("child2");
+    assertTrue("Queue 'child2' should still be 'fair' there is a policy"
+            + " violation while reinitialization.",
+        child2.getPolicy() instanceof FairSharePolicy);
+
+    // Set both child1 and root to 'drf', then reload the allocation file
+    out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"root\">");
+    out.println("<schedulingPolicy>drf</schedulingPolicy>");
+    out.println("    <queue name=\"child1\">");
+    out.println("    <schedulingPolicy>drf</schedulingPolicy>");
+    out.println("    </queue>");
+    out.println("    <queue name=\"child2\">");
+    out.println("    <schedulingPolicy>fifo</schedulingPolicy>");
+    out.println("    </queue>");
+    out.println("</queue>");
+    out.println("</allocations>");
+    out.close();
+
+    scheduler.reinitialize(conf, null);
+
+    child1 = scheduler.getQueueManager().getQueue("child1");
+    assertTrue("Queue 'child1' should be 'drf' since both 'root' and 'child1'"
+            + " are 'drf'.",
+        child1.getPolicy() instanceof DominantResourceFairnessPolicy);
+    child2 = scheduler.getQueueManager().getQueue("child2");
+    assertTrue("Queue 'child2' should still be 'fifo' there is no policy"
+            + " violation while reinitialization.",
+        child2.getPolicy() instanceof FifoPolicy);
+  }
 }


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


[33/50] [abbrv] hadoop git commit: HADOOP-14040. Use shaded aws-sdk uber-JAR 1.11.86. Contributed by Steve Loughran and Sean Mackrory

Posted by xg...@apache.org.
HADOOP-14040. Use shaded aws-sdk uber-JAR 1.11.86. Contributed by Steve Loughran and Sean Mackrory


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

Branch: refs/heads/YARN-5734
Commit: 658702efffdf52cf5ddf8e92f959f1157c95a348
Parents: bdad8b7
Author: Mingliang Liu <li...@apache.org>
Authored: Thu Feb 16 16:51:03 2017 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Thu Feb 16 16:51:03 2017 -0800

----------------------------------------------------------------------
 hadoop-project/pom.xml          | 9 ++-------
 hadoop-tools/hadoop-aws/pom.xml | 7 +------
 2 files changed, 3 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/658702ef/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 606f7fc..47e21d8 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -126,7 +126,7 @@
     <make-maven-plugin.version>1.0-beta-1</make-maven-plugin.version>
     <native-maven-plugin.version>1.0-alpha-8</native-maven-plugin.version>
     <surefire.fork.timeout>900</surefire.fork.timeout>
-    <aws-java-sdk.version>1.11.45</aws-java-sdk.version>
+    <aws-java-sdk.version>1.11.86</aws-java-sdk.version>
     <!-- the version of Hadoop declared in the version resources; can be overridden
     so that Hadoop 3.x can declare itself a 2.x artifact. -->
     <declared.hadoop.version>${project.version}</declared.hadoop.version>
@@ -791,12 +791,7 @@
       </dependency>
       <dependency>
         <groupId>com.amazonaws</groupId>
-        <artifactId>aws-java-sdk-s3</artifactId>
-        <version>${aws-java-sdk.version}</version>
-      </dependency>
-      <dependency>
-        <groupId>com.amazonaws</groupId>
-        <artifactId>aws-java-sdk-sts</artifactId>
+        <artifactId>aws-java-sdk-bundle</artifactId>
         <version>${aws-java-sdk.version}</version>
       </dependency>
       <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658702ef/hadoop-tools/hadoop-aws/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml
index 1f64b02..0fdbc5d 100644
--- a/hadoop-tools/hadoop-aws/pom.xml
+++ b/hadoop-tools/hadoop-aws/pom.xml
@@ -331,7 +331,7 @@
     </dependency>
     <dependency>
       <groupId>com.amazonaws</groupId>
-      <artifactId>aws-java-sdk-s3</artifactId>
+      <artifactId>aws-java-sdk-bundle</artifactId>
       <scope>compile</scope>
     </dependency>
     <dependency>
@@ -355,11 +355,6 @@
       <artifactId>joda-time</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.amazonaws</groupId>
-      <artifactId>aws-java-sdk-sts</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>


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


[25/50] [abbrv] hadoop git commit: YARN-6200. addendum to fix compilation error caused by reverting YARN-5068.

Posted by xg...@apache.org.
YARN-6200. addendum to fix compilation error caused by reverting YARN-5068.


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

Branch: refs/heads/YARN-5734
Commit: e63a7814d21c6469adb01a3a93cfb3ed7613437d
Parents: a393e84
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Thu Feb 16 18:19:57 2017 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Thu Feb 16 18:19:57 2017 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java   | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e63a7814/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.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/amlauncher/AMLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
index 05f9f47..7051f8c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.client.NMProxy;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;


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


[21/50] [abbrv] hadoop git commit: YARN-6163. FS Preemption is a trickle for severely starved applications. (kasha)

Posted by xg...@apache.org.
YARN-6163. FS Preemption is a trickle for severely starved applications. (kasha)


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

Branch: refs/heads/YARN-5734
Commit: 6c25dbcdc0517a825b92fb16444aa1d3761e160c
Parents: a136936
Author: Karthik Kambatla <ka...@cloudera.com>
Authored: Wed Feb 15 23:16:01 2017 -0800
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Wed Feb 15 23:16:12 2017 -0800

----------------------------------------------------------------------
 .../hadoop/yarn/util/resource/Resources.java    |  18 +++
 .../scheduler/AbstractYarnScheduler.java        |   4 +
 .../scheduler/fair/FSAppAttempt.java            | 110 ++++++++++++--
 .../scheduler/fair/FSLeafQueue.java             | 111 +++++++++-----
 .../scheduler/fair/FSPreemptionThread.java      | 132 ++++++++---------
 .../scheduler/fair/FairScheduler.java           |   4 +
 .../fair/FairSchedulerConfiguration.java        |  23 ++-
 .../fair/VisitedResourceRequestTracker.java     | 146 +++++++++++++++++++
 .../fair/FairSchedulerWithMockPreemption.java   |   5 +-
 .../scheduler/fair/TestFSAppStarvation.java     |  20 ++-
 .../fair/TestFairSchedulerPreemption.java       |  45 +++---
 .../fair/TestVisitedResourceRequestTracker.java | 112 ++++++++++++++
 12 files changed, 585 insertions(+), 145 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c25dbcd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
index 044a232..57b3a46 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
@@ -182,6 +182,24 @@ public class Resources {
     return subtractFrom(clone(lhs), rhs);
   }
 
+  /**
+   * Subtract <code>rhs</code> from <code>lhs</code> and reset any negative
+   * values to zero.
+   * @param lhs {@link Resource} to subtract from
+   * @param rhs {@link Resource} to subtract
+   * @return the value of lhs after subtraction
+   */
+  public static Resource subtractFromNonNegative(Resource lhs, Resource rhs) {
+    subtractFrom(lhs, rhs);
+    if (lhs.getMemorySize() < 0) {
+      lhs.setMemorySize(0);
+    }
+    if (lhs.getVirtualCores() < 0) {
+      lhs.setVirtualCores(0);
+    }
+    return lhs;
+  }
+
   public static Resource negate(Resource resource) {
     return subtract(NONE, resource);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c25dbcd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index 64427b7..ce6d2a2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -127,6 +127,7 @@ public abstract class AbstractYarnScheduler
    */
   protected ConcurrentMap<ApplicationId, SchedulerApplication<T>> applications;
   protected int nmExpireInterval;
+  protected long nmHeartbeatInterval;
 
   protected final static List<Container> EMPTY_CONTAINER_LIST =
       new ArrayList<Container>();
@@ -163,6 +164,9 @@ public abstract class AbstractYarnScheduler
     nmExpireInterval =
         conf.getInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS,
           YarnConfiguration.DEFAULT_RM_NM_EXPIRY_INTERVAL_MS);
+    nmHeartbeatInterval =
+        conf.getLong(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS,
+            YarnConfiguration.DEFAULT_RM_NM_HEARTBEAT_INTERVAL_MS);
     long configuredMaximumAllocationWaitTime =
         conf.getLong(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS,
           YarnConfiguration.DEFAULT_RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c25dbcd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
index 6dfcc84..563b892 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
@@ -87,6 +87,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
   private final Set<RMContainer> containersToPreempt = new HashSet<>();
   private Resource fairshareStarvation = Resources.none();
   private long lastTimeAtFairShare;
+  private long nextStarvationCheck;
 
   // minShareStarvation attributed to this application by the leaf queue
   private Resource minshareStarvation = Resources.none();
@@ -211,15 +212,9 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
       blacklistNodeIds.addAll(scheduler.getBlacklistedNodes(this));
     }
     for (FSSchedulerNode node: blacklistNodeIds) {
-      Resources.subtractFrom(availableResources,
+      Resources.subtractFromNonNegative(availableResources,
           node.getUnallocatedResource());
     }
-    if (availableResources.getMemorySize() < 0) {
-      availableResources.setMemorySize(0);
-    }
-    if (availableResources.getVirtualCores() < 0) {
-      availableResources.setVirtualCores(0);
-    }
   }
 
   /**
@@ -530,6 +525,15 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
   }
 
   /**
+   * Get last computed fairshare starvation.
+   *
+   * @return last computed fairshare starvation
+   */
+  Resource getFairshareStarvation() {
+    return fairshareStarvation;
+  }
+
+  /**
    * Set the minshare attributed to this application. To be called only from
    * {@link FSLeafQueue#updateStarvedApps}.
    *
@@ -1077,17 +1081,17 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
   }
 
   /**
-   * Helper method that computes the extent of fairshare fairshareStarvation.
+   * Helper method that computes the extent of fairshare starvation.
+   * @return freshly computed fairshare starvation
    */
   Resource fairShareStarvation() {
     Resource threshold = Resources.multiply(
         getFairShare(), fsQueue.getFairSharePreemptionThreshold());
-    Resource starvation = Resources.subtractFrom(threshold, getResourceUsage());
+    Resource starvation = Resources.componentwiseMin(threshold, demand);
+    Resources.subtractFromNonNegative(starvation, getResourceUsage());
 
     long now = scheduler.getClock().getTime();
-    boolean starved = Resources.greaterThan(
-        fsQueue.getPolicy().getResourceCalculator(),
-        scheduler.getClusterResource(), starvation, Resources.none());
+    boolean starved = !Resources.isNone(starvation);
 
     if (!starved) {
       lastTimeAtFairShare = now;
@@ -1111,6 +1115,81 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     return !Resources.isNone(fairshareStarvation);
   }
 
+  /**
+   * Fetch a list of RRs corresponding to the extent the app is starved
+   * (fairshare and minshare). This method considers the number of containers
+   * in a RR and also only one locality-level (the first encountered
+   * resourceName).
+   *
+   * @return list of {@link ResourceRequest}s corresponding to the amount of
+   * starvation.
+   */
+  List<ResourceRequest> getStarvedResourceRequests() {
+    // List of RRs we build in this method to return
+    List<ResourceRequest> ret = new ArrayList<>();
+
+    // Track visited RRs to avoid the same RR at multiple locality levels
+    VisitedResourceRequestTracker visitedRRs =
+        new VisitedResourceRequestTracker(scheduler.getNodeTracker());
+
+    // Start with current starvation and track the pending amount
+    Resource pending = getStarvation();
+    for (ResourceRequest rr : appSchedulingInfo.getAllResourceRequests()) {
+      if (Resources.isNone(pending)) {
+        // Found enough RRs to match the starvation
+        break;
+      }
+
+      // See if we have already seen this RR
+      if (!visitedRRs.visit(rr)) {
+        continue;
+      }
+
+      // A RR can have multiple containers of a capability. We need to
+      // compute the number of containers that fit in "pending".
+      int numContainersThatFit = (int) Math.floor(
+          Resources.ratio(scheduler.getResourceCalculator(),
+              pending, rr.getCapability()));
+      if (numContainersThatFit == 0) {
+        // This RR's capability is too large to fit in pending
+        continue;
+      }
+
+      // If the RR is only partially being satisfied, include only the
+      // partial number of containers.
+      if (numContainersThatFit < rr.getNumContainers()) {
+        rr = ResourceRequest.newInstance(rr.getPriority(),
+            rr.getResourceName(), rr.getCapability(), numContainersThatFit);
+      }
+
+      // Add the RR to return list and adjust "pending" accordingly
+      ret.add(rr);
+      Resources.subtractFromNonNegative(pending,
+          Resources.multiply(rr.getCapability(), rr.getNumContainers()));
+    }
+
+    return ret;
+  }
+
+  /**
+   * Notify this app that preemption has been triggered to make room for
+   * outstanding demand. The app should not be considered starved until after
+   * the specified delay.
+   *
+   * @param delayBeforeNextStarvationCheck duration to wait
+   */
+  void preemptionTriggered(long delayBeforeNextStarvationCheck) {
+    nextStarvationCheck =
+        scheduler.getClock().getTime() + delayBeforeNextStarvationCheck;
+  }
+
+  /**
+   * Whether this app's starvation should be considered.
+   */
+  boolean shouldCheckForStarvation() {
+    return scheduler.getClock().getTime() >= nextStarvationCheck;
+  }
+
   /* Schedulable methods implementation */
 
   @Override
@@ -1123,6 +1202,13 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     return demand;
   }
 
+  /**
+   * Get the current app's unsatisfied demand.
+   */
+  Resource getPendingDemand() {
+    return Resources.subtract(demand, getResourceUsage());
+  }
+
   @Override
   public long getStartTime() {
     return startTime;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c25dbcd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
index 16070e0..c4b2de6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
@@ -220,54 +220,53 @@ public class FSLeafQueue extends FSQueue {
   }
 
   /**
-   * Helper method to identify starved applications. This needs to be called
-   * ONLY from {@link #updateInternal}, after the application shares
-   * are updated.
-   *
-   * A queue can be starving due to fairshare or minshare.
-   *
-   * Minshare is defined only on the queue and not the applications.
-   * Fairshare is defined for both the queue and the applications.
-   *
-   * If this queue is starved due to minshare, we need to identify the most
-   * deserving apps if they themselves are not starved due to fairshare.
+   * Compute the extent of fairshare starvation for a set of apps.
    *
-   * If this queue is starving due to fairshare, there must be at least
-   * one application that is starved. And, even if the queue is not
-   * starved due to fairshare, there might still be starved applications.
+   * @param appsWithDemand apps to compute fairshare starvation for
+   * @return aggregate fairshare starvation for all apps
    */
-  private void updateStarvedApps() {
-    // First identify starved applications and track total amount of
-    // starvation (in resources)
+  private Resource updateStarvedAppsFairshare(
+      TreeSet<FSAppAttempt> appsWithDemand) {
     Resource fairShareStarvation = Resources.clone(none());
-
     // Fetch apps with unmet demand sorted by fairshare starvation
-    TreeSet<FSAppAttempt> appsWithDemand = fetchAppsWithDemand();
     for (FSAppAttempt app : appsWithDemand) {
       Resource appStarvation = app.fairShareStarvation();
-      if (!Resources.equals(Resources.none(), appStarvation))  {
+      if (!Resources.isNone(appStarvation))  {
         context.getStarvedApps().addStarvedApp(app);
         Resources.addTo(fairShareStarvation, appStarvation);
       } else {
         break;
       }
     }
+    return fairShareStarvation;
+  }
 
-    // Compute extent of minshare starvation
-    Resource minShareStarvation = minShareStarvation();
-
-    // Compute minshare starvation that is not subsumed by fairshare starvation
-    Resources.subtractFrom(minShareStarvation, fairShareStarvation);
+  /**
+   * Distribute minshare starvation to a set of apps
+   * @param appsWithDemand set of apps
+   * @param minShareStarvation minshare starvation to distribute
+   */
+  private void updateStarvedAppsMinshare(
+      final TreeSet<FSAppAttempt> appsWithDemand,
+      final Resource minShareStarvation) {
+    Resource pending = Resources.clone(minShareStarvation);
 
     // Keep adding apps to the starved list until the unmet demand goes over
     // the remaining minshare
     for (FSAppAttempt app : appsWithDemand) {
-      if (Resources.greaterThan(policy.getResourceCalculator(),
-          scheduler.getClusterResource(), minShareStarvation, none())) {
-        Resource appPendingDemand =
-            Resources.subtract(app.getDemand(), app.getResourceUsage());
-        Resources.subtractFrom(minShareStarvation, appPendingDemand);
-        app.setMinshareStarvation(appPendingDemand);
+      if (!Resources.isNone(pending)) {
+        Resource appMinShare = app.getPendingDemand();
+        Resources.subtractFromNonNegative(
+            appMinShare, app.getFairshareStarvation());
+
+        if (Resources.greaterThan(policy.getResourceCalculator(),
+            scheduler.getClusterResource(), appMinShare, pending)) {
+          Resources.subtractFromNonNegative(appMinShare, pending);
+          pending = none();
+        } else {
+          Resources.subtractFromNonNegative(pending, appMinShare);
+        }
+        app.setMinshareStarvation(appMinShare);
         context.getStarvedApps().addStarvedApp(app);
       } else {
         // Reset minshare starvation in case we had set it in a previous
@@ -277,6 +276,40 @@ public class FSLeafQueue extends FSQueue {
     }
   }
 
+  /**
+   * Helper method to identify starved applications. This needs to be called
+   * ONLY from {@link #updateInternal}, after the application shares
+   * are updated.
+   *
+   * A queue can be starving due to fairshare or minshare.
+   *
+   * Minshare is defined only on the queue and not the applications.
+   * Fairshare is defined for both the queue and the applications.
+   *
+   * If this queue is starved due to minshare, we need to identify the most
+   * deserving apps if they themselves are not starved due to fairshare.
+   *
+   * If this queue is starving due to fairshare, there must be at least
+   * one application that is starved. And, even if the queue is not
+   * starved due to fairshare, there might still be starved applications.
+   */
+  private void updateStarvedApps() {
+    // Fetch apps with pending demand
+    TreeSet<FSAppAttempt> appsWithDemand = fetchAppsWithDemand(false);
+
+    // Process apps with fairshare starvation
+    Resource fairShareStarvation = updateStarvedAppsFairshare(appsWithDemand);
+
+    // Compute extent of minshare starvation
+    Resource minShareStarvation = minShareStarvation();
+
+    // Compute minshare starvation that is not subsumed by fairshare starvation
+    Resources.subtractFromNonNegative(minShareStarvation, fairShareStarvation);
+
+    // Assign this minshare to apps with pending demand over fairshare
+    updateStarvedAppsMinshare(appsWithDemand, minShareStarvation);
+  }
+
   @Override
   public Resource getDemand() {
     return demand;
@@ -352,7 +385,7 @@ public class FSLeafQueue extends FSQueue {
       return assigned;
     }
 
-    for (FSAppAttempt sched : fetchAppsWithDemand()) {
+    for (FSAppAttempt sched : fetchAppsWithDemand(true)) {
       if (SchedulerAppUtils.isPlaceBlacklisted(sched, node, LOG)) {
         continue;
       }
@@ -368,14 +401,24 @@ public class FSLeafQueue extends FSQueue {
     return assigned;
   }
 
-  private TreeSet<FSAppAttempt> fetchAppsWithDemand() {
+  /**
+   * Fetch the subset of apps that have unmet demand. When used for
+   * preemption-related code (as opposed to allocation), omits apps that
+   * should not be checked for starvation.
+   *
+   * @param assignment whether the apps are for allocation containers, as
+   *                   opposed to preemption calculations
+   * @return Set of apps with unmet demand
+   */
+  private TreeSet<FSAppAttempt> fetchAppsWithDemand(boolean assignment) {
     TreeSet<FSAppAttempt> pendingForResourceApps =
         new TreeSet<>(policy.getComparator());
     readLock.lock();
     try {
       for (FSAppAttempt app : runnableApps) {
         Resource pending = app.getAppAttemptResourceUsage().getPending();
-        if (!pending.equals(none())) {
+        if (!Resources.isNone(pending) &&
+            (assignment || app.shouldCheckForStarvation())) {
           pendingForResourceApps.add(app);
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c25dbcd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSPreemptionThread.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSPreemptionThread.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSPreemptionThread.java
index f166878..af73c10 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSPreemptionThread.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSPreemptionThread.java
@@ -26,8 +26,6 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 import java.util.ArrayList;
@@ -43,20 +41,26 @@ class FSPreemptionThread extends Thread {
   protected final FSContext context;
   private final FairScheduler scheduler;
   private final long warnTimeBeforeKill;
+  private final long delayBeforeNextStarvationCheck;
   private final Timer preemptionTimer;
 
   FSPreemptionThread(FairScheduler scheduler) {
+    setDaemon(true);
+    setName("FSPreemptionThread");
     this.scheduler = scheduler;
     this.context = scheduler.getContext();
     FairSchedulerConfiguration fsConf = scheduler.getConf();
     context.setPreemptionEnabled();
     context.setPreemptionUtilizationThreshold(
         fsConf.getPreemptionUtilizationThreshold());
-    warnTimeBeforeKill = fsConf.getWaitTimeBeforeKill();
     preemptionTimer = new Timer("Preemption Timer", true);
 
-    setDaemon(true);
-    setName("FSPreemptionThread");
+    warnTimeBeforeKill = fsConf.getWaitTimeBeforeKill();
+    long allocDelay = (fsConf.isContinuousSchedulingEnabled()
+        ? 10 * fsConf.getContinuousSchedulingSleepMs() // 10 runs
+        : 4 * scheduler.getNMHeartbeatInterval()); // 4 heartbeats
+    delayBeforeNextStarvationCheck = warnTimeBeforeKill + allocDelay +
+        fsConf.getWaitTimeBeforeNextStarvationCheck();
   }
 
   public void run() {
@@ -64,13 +68,8 @@ class FSPreemptionThread extends Thread {
       FSAppAttempt starvedApp;
       try{
         starvedApp = context.getStarvedApps().take();
-        if (!Resources.isNone(starvedApp.getStarvation())) {
-          PreemptableContainers containers =
-              identifyContainersToPreempt(starvedApp);
-          if (containers != null) {
-            preemptContainers(containers.containers);
-          }
-        }
+        preemptContainers(identifyContainersToPreempt(starvedApp));
+        starvedApp.preemptionTriggered(delayBeforeNextStarvationCheck);
       } catch (InterruptedException e) {
         LOG.info("Preemption thread interrupted! Exiting.");
         return;
@@ -79,58 +78,57 @@ class FSPreemptionThread extends Thread {
   }
 
   /**
-   * Given an app, identify containers to preempt to satisfy the app's next
-   * resource request.
+   * Given an app, identify containers to preempt to satisfy the app's
+   * starvation.
+   *
+   * Mechanics:
+   * 1. Fetch all {@link ResourceRequest}s corresponding to the amount of
+   * starvation.
+   * 2. For each {@link ResourceRequest}, iterate through matching
+   * nodes and identify containers to preempt all on one node, also
+   * optimizing for least number of AM container preemptions.
    *
    * @param starvedApp starved application for which we are identifying
    *                   preemption targets
-   * @return list of containers to preempt to satisfy starvedApp, null if the
-   * app cannot be satisfied by preempting any running containers
+   * @return list of containers to preempt to satisfy starvedApp
    */
-  private PreemptableContainers identifyContainersToPreempt(
+  private List<RMContainer> identifyContainersToPreempt(
       FSAppAttempt starvedApp) {
-    PreemptableContainers bestContainers = null;
-
-    // Find the nodes that match the next resource request
-    SchedulingPlacementSet nextPs =
-        starvedApp.getAppSchedulingInfo().getFirstSchedulingPlacementSet();
-    PendingAsk firstPendingAsk = nextPs.getPendingAsk(ResourceRequest.ANY);
-    // TODO (KK): Should we check other resource requests if we can't match
-    // the first one?
-
-    Resource requestCapability = firstPendingAsk.getPerAllocationResource();
-
-    List<FSSchedulerNode> potentialNodes =
-        scheduler.getNodeTracker().getNodesByResourceName(
-            nextPs.getAcceptedResouceNames().next().toString());
+    List<RMContainer> containersToPreempt = new ArrayList<>();
+
+    // Iterate through enough RRs to address app's starvation
+    for (ResourceRequest rr : starvedApp.getStarvedResourceRequests()) {
+      for (int i = 0; i < rr.getNumContainers(); i++) {
+        PreemptableContainers bestContainers = null;
+        List<FSSchedulerNode> potentialNodes = scheduler.getNodeTracker()
+            .getNodesByResourceName(rr.getResourceName());
+        for (FSSchedulerNode node : potentialNodes) {
+          // TODO (YARN-5829): Attempt to reserve the node for starved app.
+          if (isNodeAlreadyReserved(node, starvedApp)) {
+            continue;
+          }
 
-    // From the potential nodes, pick a node that has enough containers
-    // from apps over their fairshare
-    for (FSSchedulerNode node : potentialNodes) {
-      // TODO (YARN-5829): Attempt to reserve the node for starved app. The
-      // subsequent if-check needs to be reworked accordingly.
-      FSAppAttempt nodeReservedApp = node.getReservedAppSchedulable();
-      if (nodeReservedApp != null && !nodeReservedApp.equals(starvedApp)) {
-        // This node is already reserved by another app. Let us not consider
-        // this for preemption.
-        continue;
-      }
+          int maxAMContainers = bestContainers == null ?
+              Integer.MAX_VALUE : bestContainers.numAMContainers;
+          PreemptableContainers preemptableContainers =
+              identifyContainersToPreemptOnNode(
+                  rr.getCapability(), node, maxAMContainers);
+          if (preemptableContainers != null) {
+            // This set is better than any previously identified set.
+            bestContainers = preemptableContainers;
+            if (preemptableContainers.numAMContainers == 0) {
+              break;
+            }
+          }
+        } // End of iteration through nodes for one RR
 
-      int maxAMContainers = bestContainers == null ?
-          Integer.MAX_VALUE : bestContainers.numAMContainers;
-      PreemptableContainers preemptableContainers =
-          identifyContainersToPreemptOnNode(requestCapability, node,
-              maxAMContainers);
-      if (preemptableContainers != null) {
-        if (preemptableContainers.numAMContainers == 0) {
-          return preemptableContainers;
-        } else {
-          bestContainers = preemptableContainers;
+        if (bestContainers != null && bestContainers.containers.size() > 0) {
+          containersToPreempt.addAll(bestContainers.containers);
+          trackPreemptionsAgainstNode(bestContainers.containers);
         }
       }
-    }
-
-    return bestContainers;
+    } // End of iteration over RRs
+    return containersToPreempt;
   }
 
   /**
@@ -181,23 +179,25 @@ class FSPreemptionThread extends Thread {
     return null;
   }
 
-  private void preemptContainers(List<RMContainer> containers) {
-    // Mark the containers as being considered for preemption on the node.
-    // Make sure the containers are subsequently removed by calling
-    // FSSchedulerNode#removeContainerForPreemption.
-    if (containers.size() > 0) {
-      FSSchedulerNode node = (FSSchedulerNode) scheduler.getNodeTracker()
-          .getNode(containers.get(0).getNodeId());
-      node.addContainersForPreemption(containers);
-    }
+  private boolean isNodeAlreadyReserved(
+      FSSchedulerNode node, FSAppAttempt app) {
+    FSAppAttempt nodeReservedApp = node.getReservedAppSchedulable();
+    return nodeReservedApp != null && !nodeReservedApp.equals(app);
+  }
 
+  private void trackPreemptionsAgainstNode(List<RMContainer> containers) {
+    FSSchedulerNode node = (FSSchedulerNode) scheduler.getNodeTracker()
+        .getNode(containers.get(0).getNodeId());
+    node.addContainersForPreemption(containers);
+  }
+
+  private void preemptContainers(List<RMContainer> containers) {
     // Warn application about containers to be killed
     for (RMContainer container : containers) {
       ApplicationAttemptId appAttemptId = container.getApplicationAttemptId();
       FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
-      FSLeafQueue queue = app.getQueue();
       LOG.info("Preempting container " + container +
-          " from queue " + queue.getName());
+          " from queue " + app.getQueueName());
       app.trackContainerForPreemption(container);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c25dbcd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index 134efff..18806bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -1774,4 +1774,8 @@ public class FairScheduler extends
   public float getReservableNodesRatio() {
     return reservableNodesRatio;
   }
+
+  long getNMHeartbeatInterval() {
+    return nmHeartbeatInterval;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c25dbcd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java
index b18dd7d..8e8e37b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java
@@ -114,12 +114,24 @@ public class FairSchedulerConfiguration extends Configuration {
   protected static final String PREEMPTION_THRESHOLD =
       CONF_PREFIX + "preemption.cluster-utilization-threshold";
   protected static final float DEFAULT_PREEMPTION_THRESHOLD = 0.8f;
-  
-  protected static final String PREEMPTION_INTERVAL = CONF_PREFIX + "preemptionInterval";
-  protected static final int DEFAULT_PREEMPTION_INTERVAL = 5000;
+
   protected static final String WAIT_TIME_BEFORE_KILL = CONF_PREFIX + "waitTimeBeforeKill";
   protected static final int DEFAULT_WAIT_TIME_BEFORE_KILL = 15000;
 
+  /**
+   * Configurable delay (ms) before an app's starvation is considered after
+   * it is identified. This is to give the scheduler enough time to
+   * allocate containers post preemption. This delay is added to the
+   * {@link #WAIT_TIME_BEFORE_KILL} and enough heartbeats.
+   *
+   * This is intended to be a backdoor on production clusters, and hence
+   * intentionally not documented.
+   */
+  protected static final String WAIT_TIME_BEFORE_NEXT_STARVATION_CHECK_MS =
+      CONF_PREFIX + "waitTimeBeforeNextStarvationCheck";
+  protected static final long
+      DEFAULT_WAIT_TIME_BEFORE_NEXT_STARVATION_CHECK_MS = 10000;
+
   /** Whether to assign multiple containers in one check-in. */
   public static final String  ASSIGN_MULTIPLE = CONF_PREFIX + "assignmultiple";
   protected static final boolean DEFAULT_ASSIGN_MULTIPLE = false;
@@ -251,8 +263,9 @@ public class FairSchedulerConfiguration extends Configuration {
     		"/tmp/")).getAbsolutePath() + File.separator + "fairscheduler");
   }
   
-  public int getPreemptionInterval() {
-    return getInt(PREEMPTION_INTERVAL, DEFAULT_PREEMPTION_INTERVAL);
+  public long getWaitTimeBeforeNextStarvationCheck() {
+    return getLong(WAIT_TIME_BEFORE_NEXT_STARVATION_CHECK_MS,
+        DEFAULT_WAIT_TIME_BEFORE_NEXT_STARVATION_CHECK_MS);
   }
   
   public int getWaitTimeBeforeKill() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c25dbcd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/VisitedResourceRequestTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/VisitedResourceRequestTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/VisitedResourceRequestTracker.java
new file mode 100644
index 0000000..f157263
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/VisitedResourceRequestTracker.java
@@ -0,0 +1,146 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ClusterNodeTracker;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Applications place {@link ResourceRequest}s at multiple levels. This is a
+ * helper class that allows tracking if a {@link ResourceRequest} has been
+ * visited at a different locality level.
+ *
+ * This is implemented for {@link FSAppAttempt#getStarvedResourceRequests()}.
+ * The implementation is not thread-safe.
+ */
+class VisitedResourceRequestTracker {
+  private static final Log LOG =
+      LogFactory.getLog(VisitedResourceRequestTracker.class);
+  private final Map<Priority, Map<Resource, TrackerPerPriorityResource>> map =
+      new HashMap<>();
+  private final ClusterNodeTracker<FSSchedulerNode> nodeTracker;
+
+  VisitedResourceRequestTracker(
+      ClusterNodeTracker<FSSchedulerNode> nodeTracker) {
+    this.nodeTracker = nodeTracker;
+  }
+
+  /**
+   * Check if the {@link ResourceRequest} is visited before, and track it.
+   * @param rr {@link ResourceRequest} to visit
+   * @return true if <code>rr</code> is the first visit across all
+   * locality levels, false otherwise
+   */
+  boolean visit(ResourceRequest rr) {
+    Priority priority = rr.getPriority();
+    Resource capability = rr.getCapability();
+
+    Map<Resource, TrackerPerPriorityResource> subMap = map.get(priority);
+    if (subMap == null) {
+      subMap = new HashMap<>();
+      map.put(priority, subMap);
+    }
+
+    TrackerPerPriorityResource tracker = subMap.get(capability);
+    if (tracker == null) {
+      tracker = new TrackerPerPriorityResource();
+      subMap.put(capability, tracker);
+    }
+
+    return tracker.visit(rr.getResourceName());
+  }
+
+  private class TrackerPerPriorityResource {
+    private Set<String> racksWithNodesVisited = new HashSet<>();
+    private Set<String> racksVisted = new HashSet<>();
+    private boolean anyVisited;
+
+    private boolean visitAny() {
+      if (racksVisted.isEmpty() && racksWithNodesVisited.isEmpty()) {
+        anyVisited = true;
+      }
+      return anyVisited;
+    }
+
+    private boolean visitRack(String rackName) {
+      if (anyVisited || racksWithNodesVisited.contains(rackName)) {
+        return false;
+      } else {
+        racksVisted.add(rackName);
+        return true;
+      }
+    }
+
+    private boolean visitNode(String rackName) {
+      if (anyVisited || racksVisted.contains(rackName)) {
+        return false;
+      } else {
+        racksWithNodesVisited.add(rackName);
+        return true;
+      }
+    }
+
+    /**
+     * Based on whether <code>resourceName</code> is a node, rack or ANY,
+     * check if this has been visited earlier.
+     *
+     * A node is considered visited if its rack or ANY have been visited.
+     * A rack is considered visited if any nodes or ANY have been visited.
+     * Any is considered visited if any of the nodes/racks have been visited.
+     *
+     * @param resourceName nodename or rackname or ANY
+     * @return true if this is the first visit, false otherwise
+     */
+    private boolean visit(String resourceName) {
+      if (resourceName.equals(ResourceRequest.ANY)) {
+        return visitAny();
+      }
+
+      List<FSSchedulerNode> nodes =
+          nodeTracker.getNodesByResourceName(resourceName);
+      int numNodes = nodes.size();
+      if (numNodes == 0) {
+        LOG.error("Found ResourceRequest for a non-existent node/rack named " +
+            resourceName);
+        return false;
+      }
+
+      if (numNodes == 1) {
+        // Found a single node. To be safe, let us verify it is a node and
+        // not a rack with a single node.
+        FSSchedulerNode node = nodes.get(0);
+        if (node.getNodeName().equals(resourceName)) {
+          return visitNode(node.getRackName());
+        }
+      }
+
+      // At this point, it is not ANY or a node. Must be a rack
+      return visitRack(resourceName);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c25dbcd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerWithMockPreemption.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerWithMockPreemption.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerWithMockPreemption.java
index 25780cd..706cdc9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerWithMockPreemption.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerWithMockPreemption.java
@@ -21,6 +21,8 @@ import java.util.HashSet;
 import java.util.Set;
 
 public class FairSchedulerWithMockPreemption extends FairScheduler {
+  static final long DELAY_FOR_NEXT_STARVATION_CHECK_MS = 10 * 60 * 1000;
+
   @Override
   protected void createPreemptionThread() {
     preemptionThread = new MockPreemptionThread(this);
@@ -30,7 +32,7 @@ public class FairSchedulerWithMockPreemption extends FairScheduler {
     private Set<FSAppAttempt> appsAdded = new HashSet<>();
     private int totalAppsAdded = 0;
 
-    MockPreemptionThread(FairScheduler scheduler) {
+    private MockPreemptionThread(FairScheduler scheduler) {
       super(scheduler);
     }
 
@@ -41,6 +43,7 @@ public class FairSchedulerWithMockPreemption extends FairScheduler {
           FSAppAttempt app = context.getStarvedApps().take();
           appsAdded.add(app);
           totalAppsAdded++;
+          app.preemptionTriggered(DELAY_FOR_NEXT_STARVATION_CHECK_MS);
         } catch (InterruptedException e) {
           return;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c25dbcd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppStarvation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppStarvation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppStarvation.java
index a5b2d86..3a79ac0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppStarvation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppStarvation.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
+import org.apache.hadoop.yarn.util.ControlledClock;
 
 import org.junit.After;
 import static org.junit.Assert.assertEquals;
@@ -43,6 +44,8 @@ public class TestFSAppStarvation extends FairSchedulerTestBase {
 
   private static final File ALLOC_FILE = new File(TEST_DIR, "test-QUEUES");
 
+  private final ControlledClock clock = new ControlledClock();
+
   // Node Capacity = NODE_CAPACITY_MULTIPLE * (1 GB or 1 vcore)
   private static final int NODE_CAPACITY_MULTIPLE = 4;
   private static final String[] QUEUES =
@@ -99,11 +102,17 @@ public class TestFSAppStarvation extends FairSchedulerTestBase {
             + "minshare and fairshare queues",
         3, preemptionThread.uniqueAppsAdded());
 
-    // Verify the apps get added again on a subsequent update
+    // Verify apps are added again only after the set delay for starvation has
+    // passed.
+    clock.tickSec(1);
     scheduler.update();
-    Thread.yield();
-
+    assertEquals("Apps re-added even before starvation delay passed",
+        preemptionThread.totalAppsAdded(), preemptionThread.uniqueAppsAdded());
     verifyLeafQueueStarvation();
+
+    clock.tickMsec(
+        FairSchedulerWithMockPreemption.DELAY_FOR_NEXT_STARVATION_CHECK_MS);
+    scheduler.update();
     assertTrue("Each app is marked as starved exactly once",
         preemptionThread.totalAppsAdded() > preemptionThread.uniqueAppsAdded());
   }
@@ -141,7 +150,7 @@ public class TestFSAppStarvation extends FairSchedulerTestBase {
     sendEnoughNodeUpdatesToAssignFully();
 
     // Sleep to hit the preemption timeouts
-    Thread.sleep(10);
+    clock.tickMsec(10);
 
     // Scheduler update to populate starved apps
     scheduler.update();
@@ -208,8 +217,9 @@ public class TestFSAppStarvation extends FairSchedulerTestBase {
         ALLOC_FILE.exists());
 
     resourceManager = new MockRM(conf);
-    resourceManager.start();
     scheduler = (FairScheduler) resourceManager.getResourceScheduler();
+    scheduler.setClock(clock);
+    resourceManager.start();
     preemptionThread = (FairSchedulerWithMockPreemption.MockPreemptionThread)
         scheduler.preemptionThread;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c25dbcd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
index 16df1ed..a4d69bf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerPreemption.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
+import org.apache.hadoop.yarn.util.ControlledClock;
 import org.junit.After;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -49,6 +50,9 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
   private static final File ALLOC_FILE = new File(TEST_DIR, "test-queues");
   private static final int GB = 1024;
 
+  // Scheduler clock
+  private final ControlledClock clock = new ControlledClock();
+
   // Node Capacity = NODE_CAPACITY_MULTIPLE * (1 GB or 1 vcore)
   private static final int NODE_CAPACITY_MULTIPLE = 4;
 
@@ -60,25 +64,28 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
   // Starving app that is expected to instigate preemption
   private FSAppAttempt starvingApp;
 
-  @Parameterized.Parameters
-  public static Collection<Boolean[]> getParameters() {
-    return Arrays.asList(new Boolean[][] {
-        {true}, {false}});
+  @Parameterized.Parameters(name = "{0}")
+  public static Collection<Object[]> getParameters() {
+    return Arrays.asList(new Object[][] {
+        {"FairSharePreemption", true},
+        {"MinSharePreemption", false}});
   }
 
-  public TestFairSchedulerPreemption(Boolean fairshare) throws IOException {
+  public TestFairSchedulerPreemption(String name, boolean fairshare)
+      throws IOException {
     fairsharePreemption = fairshare;
     writeAllocFile();
   }
 
   @Before
-  public void setup() {
+  public void setup() throws IOException {
     createConfiguration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE,
         ALLOC_FILE.getAbsolutePath());
     conf.setBoolean(FairSchedulerConfiguration.PREEMPTION, true);
     conf.setFloat(FairSchedulerConfiguration.PREEMPTION_THRESHOLD, 0f);
     conf.setInt(FairSchedulerConfiguration.WAIT_TIME_BEFORE_KILL, 0);
+    setupCluster();
   }
 
   @After
@@ -166,8 +173,9 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
 
   private void setupCluster() throws IOException {
     resourceManager = new MockRM(conf);
-    resourceManager.start();
     scheduler = (FairScheduler) resourceManager.getResourceScheduler();
+    scheduler.setClock(clock);
+    resourceManager.start();
 
     // Create and add two nodes to the cluster
     addNode(NODE_CAPACITY_MULTIPLE * GB, NODE_CAPACITY_MULTIPLE);
@@ -197,7 +205,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
    *
    * @param queueName queue name
    */
-  private void takeAllResource(String queueName) {
+  private void takeAllResources(String queueName) {
     // Create an app that takes up all the resources on the cluster
     ApplicationAttemptId appAttemptId
         = createSchedulingRequest(GB, 1, queueName, "default",
@@ -227,8 +235,8 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
         NODE_CAPACITY_MULTIPLE * rmNodes.size() / 2);
     starvingApp = scheduler.getSchedulerApp(appAttemptId);
 
-    // Sleep long enough to pass
-    Thread.sleep(10);
+    // Move clock enough to identify starvation
+    clock.tickSec(1);
     scheduler.update();
   }
 
@@ -243,14 +251,13 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
    */
   private void submitApps(String queue1, String queue2)
       throws InterruptedException {
-    takeAllResource(queue1);
+    takeAllResources(queue1);
     preemptHalfResources(queue2);
   }
 
   private void verifyPreemption() throws InterruptedException {
-    // Sleep long enough for four containers to be preempted. Note that the
-    // starved app must be queued four times for containers to be preempted.
-    for (int i = 0; i < 10000; i++) {
+    // Sleep long enough for four containers to be preempted.
+    for (int i = 0; i < 100; i++) {
       if (greedyApp.getLiveContainers().size() == 4) {
         break;
       }
@@ -268,7 +275,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
 
   private void verifyNoPreemption() throws InterruptedException {
     // Sleep long enough to ensure not even one container is preempted.
-    for (int i = 0; i < 600; i++) {
+    for (int i = 0; i < 100; i++) {
       if (greedyApp.getLiveContainers().size() != 8) {
         break;
       }
@@ -279,7 +286,6 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
 
   @Test
   public void testPreemptionWithinSameLeafQueue() throws Exception {
-    setupCluster();
     String queue = "root.preemptable.child-1";
     submitApps(queue, queue);
     if (fairsharePreemption) {
@@ -291,21 +297,18 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
 
   @Test
   public void testPreemptionBetweenTwoSiblingLeafQueues() throws Exception {
-    setupCluster();
     submitApps("root.preemptable.child-1", "root.preemptable.child-2");
     verifyPreemption();
   }
 
   @Test
   public void testPreemptionBetweenNonSiblingQueues() throws Exception {
-    setupCluster();
     submitApps("root.preemptable.child-1", "root.nonpreemptable.child-1");
     verifyPreemption();
   }
 
   @Test
   public void testNoPreemptionFromDisallowedQueue() throws Exception {
-    setupCluster();
     submitApps("root.nonpreemptable.child-1", "root.preemptable.child-1");
     verifyNoPreemption();
   }
@@ -331,9 +334,7 @@ public class TestFairSchedulerPreemption extends FairSchedulerTestBase {
 
   @Test
   public void testPreemptionSelectNonAMContainer() throws Exception {
-    setupCluster();
-
-    takeAllResource("root.preemptable.child-1");
+    takeAllResources("root.preemptable.child-1");
     setNumAMContainersPerNode(2);
     preemptHalfResources("root.preemptable.child-2");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c25dbcd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestVisitedResourceRequestTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestVisitedResourceRequestTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestVisitedResourceRequestTracker.java
new file mode 100644
index 0000000..07b8498
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestVisitedResourceRequestTracker.java
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at*
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
+
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ClusterNodeTracker;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Assert;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import org.junit.Test;
+
+import java.util.List;
+
+public class TestVisitedResourceRequestTracker {
+  private final ClusterNodeTracker<FSSchedulerNode>
+      nodeTracker = new ClusterNodeTracker<>();
+  private final ResourceRequest
+      anyRequest, rackRequest, node1Request, node2Request;
+
+  private final String NODE_VISITED = "The node is already visited. ";
+  private final String RACK_VISITED = "The rack is already visited. ";
+  private final String ANY_VISITED = "ANY is already visited. ";
+  private final String NODE_FAILURE = "The node is visited again.";
+  private final String RACK_FAILURE = "The rack is visited again.";
+  private final String ANY_FAILURE = "ANY is visited again.";
+  private final String FIRST_CALL_FAILURE = "First call to visit failed.";
+
+  public TestVisitedResourceRequestTracker() {
+    List<RMNode> rmNodes =
+        MockNodes.newNodes(1, 2, Resources.createResource(8192, 8));
+
+    FSSchedulerNode node1 = new FSSchedulerNode(rmNodes.get(0), false);
+    nodeTracker.addNode(node1);
+    node1Request = createRR(node1.getNodeName(), 1);
+
+    FSSchedulerNode node2 = new FSSchedulerNode(rmNodes.get(1), false);
+    node2Request = createRR(node2.getNodeName(), 1);
+    nodeTracker.addNode(node2);
+
+    anyRequest = createRR(ResourceRequest.ANY, 2);
+    rackRequest = createRR(node1.getRackName(), 2);
+  }
+
+  private ResourceRequest createRR(String resourceName, int count) {
+    return ResourceRequest.newInstance(
+        Priority.UNDEFINED, resourceName, Resources.none(), count);
+  }
+
+  @Test
+  public void testVisitAnyRequestFirst() {
+    VisitedResourceRequestTracker tracker =
+        new VisitedResourceRequestTracker(nodeTracker);
+
+    // Visit ANY request first
+    assertTrue(FIRST_CALL_FAILURE, tracker.visit(anyRequest));
+
+    // All other requests should return false
+    assertFalse(ANY_VISITED + RACK_FAILURE, tracker.visit(rackRequest));
+    assertFalse(ANY_VISITED + NODE_FAILURE, tracker.visit(node1Request));
+    assertFalse(ANY_VISITED + NODE_FAILURE, tracker.visit(node2Request));
+  }
+
+  @Test
+  public void testVisitRackRequestFirst() {
+    VisitedResourceRequestTracker tracker =
+        new VisitedResourceRequestTracker(nodeTracker);
+
+    // Visit rack request first
+    assertTrue(FIRST_CALL_FAILURE, tracker.visit(rackRequest));
+
+    // All other requests should return false
+    assertFalse(RACK_VISITED + ANY_FAILURE, tracker.visit(anyRequest));
+    assertFalse(RACK_VISITED + NODE_FAILURE, tracker.visit(node1Request));
+    assertFalse(RACK_VISITED + NODE_FAILURE, tracker.visit(node2Request));
+  }
+
+  @Test
+  public void testVisitNodeRequestFirst() {
+    VisitedResourceRequestTracker tracker =
+        new VisitedResourceRequestTracker(nodeTracker);
+
+    // Visit node1 first
+    assertTrue(FIRST_CALL_FAILURE, tracker.visit(node1Request));
+
+    // Rack and ANY should return false
+    assertFalse(NODE_VISITED + ANY_FAILURE, tracker.visit(anyRequest));
+    assertFalse(NODE_VISITED + RACK_FAILURE, tracker.visit(rackRequest));
+
+    // The other node should return true
+    assertTrue(NODE_VISITED + "Different node visit failed",
+        tracker.visit(node2Request));
+  }
+}


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


[39/50] [abbrv] hadoop git commit: HADOOP-14092. Typo in hadoop-aws index.md. Contributed by John Zhuge

Posted by xg...@apache.org.
HADOOP-14092. Typo in hadoop-aws index.md. Contributed by John Zhuge

(cherry picked from commit b1c1f05b1dc997906390d653dfafb4f0d7e193c4)


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

Branch: refs/heads/YARN-5734
Commit: 3a2e30fa9fe692fe44666c78fbaa04e8469f9d17
Parents: dbbfcf7
Author: Steve Loughran <st...@apache.org>
Authored: Sat Feb 18 18:16:19 2017 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Sat Feb 18 18:17:11 2017 +0000

----------------------------------------------------------------------
 .../hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md         | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a2e30fa/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index 0ff314c..7815bcf 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -2250,7 +2250,7 @@ like `ITestS3A*` shown above, it may cause unpredictable test failures.
 ### Testing against different regions
 
 S3A can connect to different regions \u2014the tests support this. Simply
-define the target region in `contract-tests.xml` or any `auth-keys.xml`
+define the target region in `contract-test-options.xml` or any `auth-keys.xml`
 file referenced.
 
 ```xml


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


[13/50] [abbrv] hadoop git commit: YARN-6156. AM blacklisting to consider node label partition (Bibin A Chundatt via Varun Saxena)

Posted by xg...@apache.org.
YARN-6156. AM blacklisting to consider node label partition (Bibin A Chundatt via Varun Saxena)


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

Branch: refs/heads/YARN-5734
Commit: b7613e0f406fb2b9bd5b1b3c79658e801f63c587
Parents: cd3e59a
Author: Varun Saxena <va...@apache.org>
Authored: Wed Feb 15 14:48:17 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Feb 15 14:48:17 2017 +0530

----------------------------------------------------------------------
 .../server/resourcemanager/RMServerUtils.java   | 22 ++++++
 .../nodelabels/RMNodeLabelsManager.java         | 16 +++++
 .../server/resourcemanager/rmapp/RMAppImpl.java | 12 ++--
 .../rmapp/attempt/RMAppAttemptImpl.java         |  4 +-
 .../TestCapacitySchedulerNodeLabelUpdate.java   | 73 ++++++++++++++++++++
 5 files changed, 118 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7613e0f/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 224a1da..e98141b 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
@@ -60,6 +60,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt
     .RMAppAttemptState;
@@ -561,4 +562,25 @@ public class RMServerUtils {
     }
     return newApplicationTimeout;
   }
+
+  /**
+   * Get applicable Node count for AM.
+   *
+   * @param rmContext context
+   * @param conf configuration
+   * @param amreq am resource request
+   * @return applicable node count
+   */
+  public static int getApplicableNodeCountForAM(RMContext rmContext,
+      Configuration conf, ResourceRequest amreq) {
+    if (YarnConfiguration.areNodeLabelsEnabled(conf)) {
+      RMNodeLabelsManager labelManager = rmContext.getNodeLabelManager();
+      String amNodeLabelExpression = amreq.getNodeLabelExpression();
+      amNodeLabelExpression = (amNodeLabelExpression == null
+          || amNodeLabelExpression.trim().isEmpty())
+              ? RMNodeLabelsManager.NO_LABEL : amNodeLabelExpression;
+      return labelManager.getActiveNMCountPerLabel(amNodeLabelExpression);
+    }
+    return rmContext.getScheduler().getNumClusterNodes();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7613e0f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
index 5dc8392..effe422 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
@@ -350,6 +350,22 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
     }
   }
   
+  /*
+   * Get active node count based on label.
+   */
+  public int getActiveNMCountPerLabel(String label) {
+    if (label == null) {
+      return 0;
+    }
+    try {
+      readLock.lock();
+      RMNodeLabel labelInfo = labelCollections.get(label);
+      return (labelInfo == null) ? 0 : labelInfo.getNumActiveNMs();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
   public Set<String> getLabelsOnNode(NodeId nodeId) {
     try {
       readLock.lock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7613e0f/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 12ece3f..516109b 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
@@ -18,11 +18,9 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
 
-import java.io.IOException;
 import java.net.InetAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -46,9 +44,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.DataInputByteBuffer;
 import org.apache.hadoop.ipc.CallerContext;
-import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -987,9 +983,11 @@ public class RMAppImpl implements RMApp, Recoverable {
       // Transfer over the blacklist from the previous app-attempt.
       currentAMBlacklistManager = currentAttempt.getAMBlacklistManager();
     } else {
-      if (amBlacklistingEnabled) {
+      if (amBlacklistingEnabled && !submissionContext.getUnmanagedAM()) {
         currentAMBlacklistManager = new SimpleBlacklistManager(
-            scheduler.getNumClusterNodes(), blacklistDisableThreshold);
+            RMServerUtils.getApplicableNodeCountForAM(rmContext, conf,
+                getAMResourceRequest()),
+            blacklistDisableThreshold);
       } else {
         currentAMBlacklistManager = new DisabledBlacklistManager();
       }
@@ -1006,7 +1004,7 @@ public class RMAppImpl implements RMApp, Recoverable {
     attempts.put(appAttemptId, attempt);
     currentAttempt = attempt;
   }
-  
+
   private void
       createAndStartNewAttempt(boolean transferStateFromPreviousAttempt) {
     createNewAttempt();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7613e0f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
index ab84985..1788722 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
@@ -1057,7 +1057,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
         appAttempt.amReq.setRelaxLocality(true);
 
         appAttempt.getAMBlacklistManager().refreshNodeHostCount(
-            appAttempt.scheduler.getNumClusterNodes());
+            RMServerUtils.getApplicableNodeCountForAM(appAttempt.rmContext,
+                appAttempt.conf, appAttempt.amReq));
 
         ResourceBlacklistRequest amBlacklist =
             appAttempt.getAMBlacklistManager().getBlacklistUpdates();
@@ -1246,7 +1247,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     }
   }
 
-
   private void rememberTargetTransitions(RMAppAttemptEvent event,
       Object transitionToDo, RMAppAttemptState targetFinalState) {
     transitionTodo = transitionToDo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b7613e0f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.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/TestCapacitySchedulerNodeLabelUpdate.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/TestCapacitySchedulerNodeLabelUpdate.java
index 732b5d1..b4ebd15 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/TestCapacitySchedulerNodeLabelUpdate.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/TestCapacitySchedulerNodeLabelUpdate.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
@@ -40,6 +41,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 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.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -721,6 +723,77 @@ public class TestCapacitySchedulerNodeLabelUpdate {
     rm.close();
   }
 
+  @Test(timeout = 30000)
+  public void testBlacklistAMDisableLabel() throws Exception {
+    conf.setBoolean(YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_ENABLED,
+        true);
+    conf.setFloat(
+        YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD,
+        0.5f);
+    mgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y"));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h2", 0), toSet("x"),
+        NodeId.newInstance("h3", 0), toSet("x"), NodeId.newInstance("h6", 0),
+        toSet("x")));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h4", 0), toSet("y"),
+        NodeId.newInstance("h5", 0), toSet("y"), NodeId.newInstance("h7", 0),
+        toSet("y")));
+
+    MockRM rm = new MockRM(getConfigurationWithQueueLabels(conf)) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+    rm.getRMContext().setNodeLabelManager(mgr);
+    rm.start();
+    // Nodes in label default h1,h8,h9
+    // Nodes in label x h2,h3,h6
+    // Nodes in label y h4,h5,h7
+    MockNM nm1 = rm.registerNode("h1:1234", 2048);
+    MockNM nm2 = rm.registerNode("h2:1234", 2048);
+    rm.registerNode("h3:1234", 2048);
+    rm.registerNode("h4:1234", 2048);
+    rm.registerNode("h5:1234", 2048);
+    rm.registerNode("h6:1234", 2048);
+    rm.registerNode("h7:1234", 2048);
+    rm.registerNode("h8:1234", 2048);
+    rm.registerNode("h9:1234", 2048);
+
+    // Submit app with AM container launched on default partition i.e. h1.
+    RMApp app = rm.submitApp(GB, "app", "user", null, "a");
+    MockRM.launchAndRegisterAM(app, rm, nm1);
+    RMAppAttempt appAttempt = app.getCurrentAppAttempt();
+    // Add default node blacklist from default
+    appAttempt.getAMBlacklistManager().addNode("h1");
+    ResourceBlacklistRequest blacklistUpdates =
+        appAttempt.getAMBlacklistManager().getBlacklistUpdates();
+    Assert.assertEquals(1, blacklistUpdates.getBlacklistAdditions().size());
+    Assert.assertEquals(0, blacklistUpdates.getBlacklistRemovals().size());
+    // Adding second node from default parition
+    appAttempt.getAMBlacklistManager().addNode("h8");
+    blacklistUpdates = appAttempt.getAMBlacklistManager().getBlacklistUpdates();
+    Assert.assertEquals(0, blacklistUpdates.getBlacklistAdditions().size());
+    Assert.assertEquals(2, blacklistUpdates.getBlacklistRemovals().size());
+
+    // Submission in label x
+    RMApp applabel = rm.submitApp(GB, "app", "user", null, "a", "x");
+    MockRM.launchAndRegisterAM(applabel, rm, nm2);
+    RMAppAttempt appAttemptlabelx = applabel.getCurrentAppAttempt();
+    appAttemptlabelx.getAMBlacklistManager().addNode("h2");
+    ResourceBlacklistRequest blacklistUpdatesOnx =
+        appAttemptlabelx.getAMBlacklistManager().getBlacklistUpdates();
+    Assert.assertEquals(1, blacklistUpdatesOnx.getBlacklistAdditions().size());
+    Assert.assertEquals(0, blacklistUpdatesOnx.getBlacklistRemovals().size());
+    // Adding second node from default parition
+    appAttemptlabelx.getAMBlacklistManager().addNode("h3");
+    blacklistUpdatesOnx =
+        appAttempt.getAMBlacklistManager().getBlacklistUpdates();
+    Assert.assertEquals(0, blacklistUpdatesOnx.getBlacklistAdditions().size());
+    Assert.assertEquals(2, blacklistUpdatesOnx.getBlacklistRemovals().size());
+
+    rm.close();
+  }
+
   private void checkAMResourceLimit(MockRM rm, String queuename, int memory,
       String label) throws InterruptedException {
     Assert.assertEquals(memory,


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


[23/50] [abbrv] hadoop git commit: YARN-5798. Set UncaughtExceptionHandler for all FairScheduler threads. (Yufei Gu via kasha)

Posted by xg...@apache.org.
YARN-5798. Set UncaughtExceptionHandler for all FairScheduler threads. (Yufei Gu via kasha)


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

Branch: refs/heads/YARN-5734
Commit: 74dd14225059322825f706120aa57cf673820daf
Parents: 11be3f7
Author: Karthik Kambatla <ka...@cloudera.com>
Authored: Thu Feb 16 00:03:09 2017 -0800
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Thu Feb 16 00:03:09 2017 -0800

----------------------------------------------------------------------
 .../server/resourcemanager/scheduler/fair/FairScheduler.java  | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/74dd1422/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index 18806bc..c5bf02a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
 import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.RMCriticalThreadUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
@@ -1268,12 +1269,16 @@ public class FairScheduler extends
 
       updateThread = new UpdateThread();
       updateThread.setName("FairSchedulerUpdateThread");
+      updateThread.setUncaughtExceptionHandler(
+          new RMCriticalThreadUncaughtExceptionHandler(rmContext));
       updateThread.setDaemon(true);
 
       if (continuousSchedulingEnabled) {
         // start continuous scheduling thread
         schedulingThread = new ContinuousSchedulingThread();
         schedulingThread.setName("FairSchedulerContinuousScheduling");
+        schedulingThread.setUncaughtExceptionHandler(
+            new RMCriticalThreadUncaughtExceptionHandler(rmContext));
         schedulingThread.setDaemon(true);
       }
 
@@ -1299,6 +1304,8 @@ public class FairScheduler extends
   @VisibleForTesting
   protected void createPreemptionThread() {
     preemptionThread = new FSPreemptionThread(this);
+    preemptionThread.setUncaughtExceptionHandler(
+        new RMCriticalThreadUncaughtExceptionHandler(rmContext));
   }
 
   private void updateReservationThreshold() {


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


[08/50] [abbrv] hadoop git commit: YARN-6061. Addendum. Remove extraneous change.

Posted by xg...@apache.org.
YARN-6061. Addendum. Remove extraneous change.


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

Branch: refs/heads/YARN-5734
Commit: 353a9b2d9165a221491395edbadf8acc3a39990b
Parents: 8e53f2b
Author: Karthik Kambatla <ka...@cloudera.com>
Authored: Tue Feb 14 15:19:52 2017 -0800
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Tue Feb 14 15:19:52 2017 -0800

----------------------------------------------------------------------
 .../scheduler/fair/policies/DominantResourceFairnessPolicy.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/353a9b2d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
index 7a29735..ad41b11 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
@@ -90,7 +90,7 @@ public class DominantResourceFairnessPolicy extends SchedulingPolicy {
 
   @Override
   public boolean checkIfUsageOverFairShare(Resource usage, Resource fairShare) {
-    return Resources.greaterThan(CALCULATOR, null, usage, fairShare);
+    return !Resources.fitsIn(usage, fairShare);
   }
 
   @Override


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


[49/50] [abbrv] hadoop git commit: HDFS-11430. Separate class InnerNode from class NetworkTopology and make it extendable. Contributed by Tsz Wo Nicholas Sze

Posted by xg...@apache.org.
HDFS-11430. Separate class InnerNode from class NetworkTopology and make it extendable. Contributed by Tsz Wo Nicholas Sze


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

Branch: refs/heads/YARN-5734
Commit: 003ae00693d079799c4dcf02705379bcf34b8c79
Parents: 8ef7ebb
Author: Mingliang Liu <li...@apache.org>
Authored: Tue Feb 21 15:29:20 2017 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Tue Feb 21 15:32:46 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/net/InnerNode.java   |  67 ++++
 .../org/apache/hadoop/net/InnerNodeImpl.java    | 304 +++++++++++++++++
 .../org/apache/hadoop/net/NetworkTopology.java  | 326 +------------------
 .../net/NetworkTopologyWithNodeGroup.java       |  43 +--
 .../apache/hadoop/net/TestNetworkTopology.java  |   2 +-
 5 files changed, 388 insertions(+), 354 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/003ae006/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/InnerNode.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/InnerNode.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/InnerNode.java
new file mode 100644
index 0000000..d07929b
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/InnerNode.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.net;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.List;
+
+
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Unstable
+public interface InnerNode extends Node {
+  interface Factory<N extends InnerNode> {
+    /** Construct an InnerNode from a path-like string */
+    N newInnerNode(String path);
+  }
+
+  /** Add node <i>n</i> to the subtree of this node
+   * @param n node to be added
+   * @return true if the node is added; false otherwise
+   */
+  boolean add(Node n);
+
+  /** Given a node's string representation, return a reference to the node
+   * @param loc string location of the form /rack/node
+   * @return null if the node is not found or the childnode is there but
+   * not an instance of {@link InnerNodeImpl}
+   */
+  Node getLoc(String loc);
+
+  /** @return its children */
+  List<Node> getChildren();
+
+  /** @return the number of leave nodes. */
+  int getNumOfLeaves();
+
+  /** Remove node <i>n</i> from the subtree of this node
+   * @param n node to be deleted
+   * @return true if the node is deleted; false otherwise
+   */
+  boolean remove(Node n);
+
+  /** get <i>leafIndex</i> leaf of this subtree
+   * if it is not in the <i>excludedNode</i>
+   *
+   * @param leafIndex an indexed leaf of the node
+   * @param excludedNode an excluded node (can be null)
+   * @return
+   */
+  Node getLeaf(int leafIndex, Node excludedNode);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/003ae006/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/InnerNodeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/InnerNodeImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/InnerNodeImpl.java
new file mode 100644
index 0000000..e6aa0f7
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/InnerNodeImpl.java
@@ -0,0 +1,304 @@
+/**
+ * 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.net;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** InnerNode represents a switch/router of a data center or rack.
+ * Different from a leaf node, it has non-null children.
+ */
+class InnerNodeImpl extends NodeBase implements InnerNode {
+  static class Factory implements InnerNode.Factory<InnerNodeImpl> {
+    private Factory() {}
+
+    @Override
+    public InnerNodeImpl newInnerNode(String path) {
+      return new InnerNodeImpl(path);
+    }
+  }
+
+  static final Factory FACTORY = new Factory();
+
+  private final List<Node> children = new ArrayList<>();
+  private final Map<String, Node> childrenMap = new HashMap<>();
+  private int numOfLeaves;
+
+  /** Construct an InnerNode from a path-like string */
+  InnerNodeImpl(String path) {
+    super(path);
+  }
+
+  /** Construct an InnerNode
+   * from its name, its network location, its parent, and its level */
+  InnerNodeImpl(String name, String location, InnerNode parent, int level) {
+    super(name, location, parent, level);
+  }
+
+  @Override
+  public List<Node> getChildren() {return children;}
+
+  /** @return the number of children this node has */
+  int getNumOfChildren() {
+    return children.size();
+  }
+
+  /** Judge if this node represents a rack
+   * @return true if it has no child or its children are not InnerNodes
+   */
+  boolean isRack() {
+    if (children.isEmpty()) {
+      return true;
+    }
+
+    Node firstChild = children.get(0);
+    if (firstChild instanceof InnerNode) {
+      return false;
+    }
+
+    return true;
+  }
+
+  /** Judge if this node is an ancestor of node <i>n</i>
+   *
+   * @param n a node
+   * @return true if this node is an ancestor of <i>n</i>
+   */
+  boolean isAncestor(Node n) {
+    return getPath(this).equals(NodeBase.PATH_SEPARATOR_STR) ||
+      (n.getNetworkLocation()+NodeBase.PATH_SEPARATOR_STR).
+      startsWith(getPath(this)+NodeBase.PATH_SEPARATOR_STR);
+  }
+
+  /** Judge if this node is the parent of node <i>n</i>
+   *
+   * @param n a node
+   * @return true if this node is the parent of <i>n</i>
+   */
+  boolean isParent(Node n) {
+    return n.getNetworkLocation().equals(getPath(this));
+  }
+
+  /* Return a child name of this node who is an ancestor of node <i>n</i> */
+  private String getNextAncestorName(Node n) {
+    if (!isAncestor(n)) {
+      throw new IllegalArgumentException(
+                                         this + "is not an ancestor of " + n);
+    }
+    String name = n.getNetworkLocation().substring(getPath(this).length());
+    if (name.charAt(0) == PATH_SEPARATOR) {
+      name = name.substring(1);
+    }
+    int index=name.indexOf(PATH_SEPARATOR);
+    if (index !=-1)
+      name = name.substring(0, index);
+    return name;
+  }
+
+  @Override
+  public boolean add(Node n) {
+    if (!isAncestor(n)) {
+      throw new IllegalArgumentException(n.getName()
+          + ", which is located at " + n.getNetworkLocation()
+          + ", is not a descendant of " + getPath(this));
+    }
+    if (isParent(n)) {
+      // this node is the parent of n; add n directly
+      n.setParent(this);
+      n.setLevel(this.level+1);
+      Node prev = childrenMap.put(n.getName(), n);
+      if (prev != null) {
+        for(int i=0; i<children.size(); i++) {
+          if (children.get(i).getName().equals(n.getName())) {
+            children.set(i, n);
+            return false;
+          }
+        }
+      }
+      children.add(n);
+      numOfLeaves++;
+      return true;
+    } else {
+      // find the next ancestor node
+      String parentName = getNextAncestorName(n);
+      InnerNode parentNode = (InnerNode)childrenMap.get(parentName);
+      if (parentNode == null) {
+        // create a new InnerNode
+        parentNode = createParentNode(parentName);
+        children.add(parentNode);
+        childrenMap.put(parentNode.getName(), parentNode);
+      }
+      // add n to the subtree of the next ancestor node
+      if (parentNode.add(n)) {
+        numOfLeaves++;
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  /**
+   * Creates a parent node to be added to the list of children.
+   * Creates a node using the InnerNode four argument constructor specifying
+   * the name, location, parent, and level of this node.
+   *
+   * <p>To be overridden in subclasses for specific InnerNode implementations,
+   * as alternative to overriding the full {@link #add(Node)} method.
+   *
+   * @param parentName The name of the parent node
+   * @return A new inner node
+   * @see InnerNodeImpl(String, String, InnerNode, int)
+   */
+  private InnerNodeImpl createParentNode(String parentName) {
+    return new InnerNodeImpl(parentName, getPath(this), this, this.getLevel()+1);
+  }
+
+  @Override
+  public boolean remove(Node n) {
+    if (!isAncestor(n)) {
+      throw new IllegalArgumentException(n.getName()
+          + ", which is located at " + n.getNetworkLocation()
+          + ", is not a descendant of " + getPath(this));
+    }
+    if (isParent(n)) {
+      // this node is the parent of n; remove n directly
+      if (childrenMap.containsKey(n.getName())) {
+        for (int i=0; i<children.size(); i++) {
+          if (children.get(i).getName().equals(n.getName())) {
+            children.remove(i);
+            childrenMap.remove(n.getName());
+            numOfLeaves--;
+            n.setParent(null);
+            return true;
+          }
+        }
+      }
+      return false;
+    } else {
+      // find the next ancestor node: the parent node
+      String parentName = getNextAncestorName(n);
+      InnerNodeImpl parentNode = (InnerNodeImpl)childrenMap.get(parentName);
+      if (parentNode == null) {
+        return false;
+      }
+      // remove n from the parent node
+      boolean isRemoved = parentNode.remove(n);
+      // if the parent node has no children, remove the parent node too
+      if (isRemoved) {
+        if (parentNode.getNumOfChildren() == 0) {
+          for(int i=0; i < children.size(); i++) {
+            if (children.get(i).getName().equals(parentName)) {
+              children.remove(i);
+              childrenMap.remove(parentName);
+              break;
+            }
+          }
+        }
+        numOfLeaves--;
+      }
+      return isRemoved;
+    }
+  } // end of remove
+
+  @Override
+  public Node getLoc(String loc) {
+    if (loc == null || loc.length() == 0) return this;
+
+    String[] path = loc.split(PATH_SEPARATOR_STR, 2);
+    Node childnode = childrenMap.get(path[0]);
+    if (childnode == null) return null; // non-existing node
+    if (path.length == 1) return childnode;
+    if (childnode instanceof InnerNode) {
+      return ((InnerNode)childnode).getLoc(path[1]);
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public Node getLeaf(int leafIndex, Node excludedNode) {
+    int count=0;
+    // check if the excluded node a leaf
+    boolean isLeaf =
+      excludedNode == null || !(excludedNode instanceof InnerNode);
+    // calculate the total number of excluded leaf nodes
+    int numOfExcludedLeaves =
+      isLeaf ? 1 : ((InnerNode)excludedNode).getNumOfLeaves();
+    if (isLeafParent()) { // children are leaves
+      if (isLeaf) { // excluded node is a leaf node
+        if (excludedNode != null &&
+            childrenMap.containsKey(excludedNode.getName())) {
+          int excludedIndex = children.indexOf(excludedNode);
+          if (excludedIndex != -1 && leafIndex >= 0) {
+            // excluded node is one of the children so adjust the leaf index
+            leafIndex = leafIndex>=excludedIndex ? leafIndex+1 : leafIndex;
+          }
+        }
+      }
+      // range check
+      if (leafIndex<0 || leafIndex>=this.getNumOfChildren()) {
+        return null;
+      }
+      return children.get(leafIndex);
+    } else {
+      for(int i=0; i<children.size(); i++) {
+        InnerNodeImpl child = (InnerNodeImpl)children.get(i);
+        if (excludedNode == null || excludedNode != child) {
+          // not the excludedNode
+          int numOfLeaves = child.getNumOfLeaves();
+          if (excludedNode != null && child.isAncestor(excludedNode)) {
+            numOfLeaves -= numOfExcludedLeaves;
+          }
+          if (count+numOfLeaves > leafIndex) {
+            // the leaf is in the child subtree
+            return child.getLeaf(leafIndex-count, excludedNode);
+          } else {
+            // go to the next child
+            count = count+numOfLeaves;
+          }
+        } else { // it is the excluededNode
+          // skip it and set the excludedNode to be null
+          excludedNode = null;
+        }
+      }
+      return null;
+    }
+  }
+
+  private boolean isLeafParent() {
+    return isRack();
+  }
+
+  @Override
+  public int getNumOfLeaves() {
+    return numOfLeaves;
+  }
+
+  @Override
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object to) {
+    return super.equals(to);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/003ae006/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
index 5751d2b..38c02f8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
@@ -17,18 +17,9 @@
  */
 package org.apache.hadoop.net;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.TreeMap;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -37,8 +28,9 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
+import java.util.*;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 /** The class represents a cluster of computer with a tree hierarchical
  * network topology.
@@ -81,314 +73,11 @@ public class NetworkTopology {
         NetworkTopology.class, NetworkTopology.class), conf);
   }
 
-  /** InnerNode represents a switch/router of a data center or rack.
-   * Different from a leaf node, it has non-null children.
-   */
-  static class InnerNode extends NodeBase {
-    protected List<Node> children=new ArrayList<Node>();
-    private Map<String, Node> childrenMap = new HashMap<String, Node>();
-    private int numOfLeaves;
-        
-    /** Construct an InnerNode from a path-like string */
-    InnerNode(String path) {
-      super(path);
-    }
-        
-    /** Construct an InnerNode from its name and its network location */
-    InnerNode(String name, String location) {
-      super(name, location);
-    }
-        
-    /** Construct an InnerNode
-     * from its name, its network location, its parent, and its level */
-    InnerNode(String name, String location, InnerNode parent, int level) {
-      super(name, location, parent, level);
-    }
-        
-    /** @return its children */
-    List<Node> getChildren() {return children;}
-        
-    /** @return the number of children this node has */
-    int getNumOfChildren() {
-      return children.size();
-    }
-        
-    /** Judge if this node represents a rack 
-     * @return true if it has no child or its children are not InnerNodes
-     */ 
-    boolean isRack() {
-      if (children.isEmpty()) {
-        return true;
-      }
-            
-      Node firstChild = children.get(0);
-      if (firstChild instanceof InnerNode) {
-        return false;
-      }
-            
-      return true;
-    }
-        
-    /** Judge if this node is an ancestor of node <i>n</i>
-     * 
-     * @param n a node
-     * @return true if this node is an ancestor of <i>n</i>
-     */
-    boolean isAncestor(Node n) {
-      return getPath(this).equals(NodeBase.PATH_SEPARATOR_STR) ||
-        (n.getNetworkLocation()+NodeBase.PATH_SEPARATOR_STR).
-        startsWith(getPath(this)+NodeBase.PATH_SEPARATOR_STR);
-    }
-        
-    /** Judge if this node is the parent of node <i>n</i>
-     * 
-     * @param n a node
-     * @return true if this node is the parent of <i>n</i>
-     */
-    boolean isParent(Node n) {
-      return n.getNetworkLocation().equals(getPath(this));
-    }
-        
-    /* Return a child name of this node who is an ancestor of node <i>n</i> */
-    private String getNextAncestorName(Node n) {
-      if (!isAncestor(n)) {
-        throw new IllegalArgumentException(
-                                           this + "is not an ancestor of " + n);
-      }
-      String name = n.getNetworkLocation().substring(getPath(this).length());
-      if (name.charAt(0) == PATH_SEPARATOR) {
-        name = name.substring(1);
-      }
-      int index=name.indexOf(PATH_SEPARATOR);
-      if (index !=-1)
-        name = name.substring(0, index);
-      return name;
-    }
-        
-    /** Add node <i>n</i> to the subtree of this node 
-     * @param n node to be added
-     * @return true if the node is added; false otherwise
-     */
-    boolean add(Node n) {
-      if (!isAncestor(n)) {
-        throw new IllegalArgumentException(n.getName()
-            + ", which is located at " + n.getNetworkLocation()
-            + ", is not a descendant of " + getPath(this));
-      }
-      if (isParent(n)) {
-        // this node is the parent of n; add n directly
-        n.setParent(this);
-        n.setLevel(this.level+1);
-        Node prev = childrenMap.put(n.getName(), n);
-        if (prev != null) {
-          for(int i=0; i<children.size(); i++) {
-            if (children.get(i).getName().equals(n.getName())) {
-              children.set(i, n);
-              return false;
-            }
-          }
-        }
-        children.add(n);
-        numOfLeaves++;
-        return true;
-      } else {
-        // find the next ancestor node
-        String parentName = getNextAncestorName(n);
-        InnerNode parentNode = (InnerNode)childrenMap.get(parentName);
-        if (parentNode == null) {
-          // create a new InnerNode
-          parentNode = createParentNode(parentName);
-          children.add(parentNode);
-          childrenMap.put(parentNode.getName(), parentNode);
-        }
-        // add n to the subtree of the next ancestor node
-        if (parentNode.add(n)) {
-          numOfLeaves++;
-          return true;
-        } else {
-          return false;
-        }
-      }
-    }
-
-    /**
-     * Creates a parent node to be added to the list of children.  
-     * Creates a node using the InnerNode four argument constructor specifying 
-     * the name, location, parent, and level of this node.
-     * 
-     * <p>To be overridden in subclasses for specific InnerNode implementations,
-     * as alternative to overriding the full {@link #add(Node)} method.
-     * 
-     * @param parentName The name of the parent node
-     * @return A new inner node
-     * @see InnerNode#InnerNode(String, String, InnerNode, int)
-     */
-    protected InnerNode createParentNode(String parentName) {
-      return new InnerNode(parentName, getPath(this), this, this.getLevel()+1);
-    }
-
-    /** Remove node <i>n</i> from the subtree of this node
-     * @param n node to be deleted 
-     * @return true if the node is deleted; false otherwise
-     */
-    boolean remove(Node n) {
-      if (!isAncestor(n)) {
-        throw new IllegalArgumentException(n.getName()
-            + ", which is located at " + n.getNetworkLocation()
-            + ", is not a descendant of " + getPath(this));
-      }
-      if (isParent(n)) {
-        // this node is the parent of n; remove n directly
-        if (childrenMap.containsKey(n.getName())) {
-          for (int i=0; i<children.size(); i++) {
-            if (children.get(i).getName().equals(n.getName())) {
-              children.remove(i);
-              childrenMap.remove(n.getName());
-              numOfLeaves--;
-              n.setParent(null);
-              return true;
-            }
-          }
-        }
-        return false;
-      } else {
-        // find the next ancestor node: the parent node
-        String parentName = getNextAncestorName(n);
-        InnerNode parentNode = (InnerNode)childrenMap.get(parentName);
-        if (parentNode == null) {
-          return false;
-        }
-        // remove n from the parent node
-        boolean isRemoved = parentNode.remove(n);
-        // if the parent node has no children, remove the parent node too
-        if (isRemoved) {
-          if (parentNode.getNumOfChildren() == 0) {
-            for(int i=0; i < children.size(); i++) {
-              if (children.get(i).getName().equals(parentName)) {
-                children.remove(i);
-                childrenMap.remove(parentName);
-                break;
-              }
-            }
-          }
-          numOfLeaves--;
-        }
-        return isRemoved;
-      }
-    } // end of remove
-        
-    /** Given a node's string representation, return a reference to the node
-     * @param loc string location of the form /rack/node
-     * @return null if the node is not found or the childnode is there but
-     * not an instance of {@link InnerNode}
-     */
-    private Node getLoc(String loc) {
-      if (loc == null || loc.length() == 0) return this;
-            
-      String[] path = loc.split(PATH_SEPARATOR_STR, 2);
-      Node childnode = childrenMap.get(path[0]);
-      if (childnode == null) return null; // non-existing node
-      if (path.length == 1) return childnode;
-      if (childnode instanceof InnerNode) {
-        return ((InnerNode)childnode).getLoc(path[1]);
-      } else {
-        return null;
-      }
-    }
-        
-    /** get <i>leafIndex</i> leaf of this subtree 
-     * if it is not in the <i>excludedNode</i>
-     *
-     * @param leafIndex an indexed leaf of the node
-     * @param excludedNode an excluded node (can be null)
-     * @return
-     */
-    Node getLeaf(int leafIndex, Node excludedNode) {
-      int count=0;
-      // check if the excluded node a leaf
-      boolean isLeaf =
-        excludedNode == null || !(excludedNode instanceof InnerNode);
-      // calculate the total number of excluded leaf nodes
-      int numOfExcludedLeaves =
-        isLeaf ? 1 : ((InnerNode)excludedNode).getNumOfLeaves();
-      if (isLeafParent()) { // children are leaves
-        if (isLeaf) { // excluded node is a leaf node
-          if (excludedNode != null &&
-              childrenMap.containsKey(excludedNode.getName())) {
-            int excludedIndex = children.indexOf(excludedNode);
-            if (excludedIndex != -1 && leafIndex >= 0) {
-              // excluded node is one of the children so adjust the leaf index
-              leafIndex = leafIndex>=excludedIndex ? leafIndex+1 : leafIndex;
-            }
-          }
-        }
-        // range check
-        if (leafIndex<0 || leafIndex>=this.getNumOfChildren()) {
-          return null;
-        }
-        return children.get(leafIndex);
-      } else {
-        for(int i=0; i<children.size(); i++) {
-          InnerNode child = (InnerNode)children.get(i);
-          if (excludedNode == null || excludedNode != child) {
-            // not the excludedNode
-            int numOfLeaves = child.getNumOfLeaves();
-            if (excludedNode != null && child.isAncestor(excludedNode)) {
-              numOfLeaves -= numOfExcludedLeaves;
-            }
-            if (count+numOfLeaves > leafIndex) {
-              // the leaf is in the child subtree
-              return child.getLeaf(leafIndex-count, excludedNode);
-            } else {
-              // go to the next child
-              count = count+numOfLeaves;
-            }
-          } else { // it is the excluededNode
-            // skip it and set the excludedNode to be null
-            excludedNode = null;
-          }
-        }
-        return null;
-      }
-    }
-    
-    protected boolean isLeafParent() {
-      return isRack();
-    }
-
-    /**
-      * Determine if children a leaves, default implementation calls {@link #isRack()}
-      * <p>To be overridden in subclasses for specific InnerNode implementations,
-      * as alternative to overriding the full {@link #getLeaf(int, Node)} method.
-      * 
-      * @return true if children are leaves, false otherwise
-      */
-    protected boolean areChildrenLeaves() {
-      return isRack();
-    }
-
-    /**
-     * Get number of leaves.
-     */
-    int getNumOfLeaves() {
-      return numOfLeaves;
-    }
-
-    @Override
-    public int hashCode() {
-      return super.hashCode();
-    }
-
-    @Override
-    public boolean equals(Object to) {
-      return super.equals(to);
-    }
-  } // end of InnerNode
-
+  InnerNode.Factory factory = InnerNodeImpl.FACTORY;
   /**
    * the root cluster map
    */
-  InnerNode clusterMap;
+  InnerNode clusterMap = factory.newInnerNode(NodeBase.ROOT);
   /** Depth of all leaf nodes */
   private int depthOfAllLeaves = -1;
   /** rack counter */
@@ -404,7 +93,6 @@ public class NetworkTopology {
   protected ReadWriteLock netlock = new ReentrantReadWriteLock();
 
   public NetworkTopology() {
-    clusterMap = new InnerNode(InnerNode.ROOT);
   }
 
   /** Add a leaf node

http://git-wip-us.apache.org/repos/asf/hadoop/blob/003ae006/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopologyWithNodeGroup.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopologyWithNodeGroup.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopologyWithNodeGroup.java
index 8ebe846..a20d5fc 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopologyWithNodeGroup.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopologyWithNodeGroup.java
@@ -36,7 +36,7 @@ public class NetworkTopologyWithNodeGroup extends NetworkTopology {
   public final static String DEFAULT_NODEGROUP = "/default-nodegroup";
 
   public NetworkTopologyWithNodeGroup() {
-    clusterMap = new InnerNodeWithNodeGroup(InnerNode.ROOT);
+    clusterMap = new InnerNodeWithNodeGroup(NodeBase.ROOT);
   }
 
   @Override
@@ -58,7 +58,7 @@ public class NetworkTopologyWithNodeGroup extends NetworkTopology {
   public String getRack(String loc) {
     netlock.readLock().lock();
     try {
-      loc = InnerNode.normalize(loc);
+      loc = NodeBase.normalize(loc);
       Node locNode = getNode(loc);
       if (locNode instanceof InnerNodeWithNodeGroup) {
         InnerNodeWithNodeGroup node = (InnerNodeWithNodeGroup) locNode;
@@ -90,7 +90,7 @@ public class NetworkTopologyWithNodeGroup extends NetworkTopology {
   public String getNodeGroup(String loc) {
     netlock.readLock().lock();
     try {
-      loc = InnerNode.normalize(loc);
+      loc = NodeBase.normalize(loc);
       Node locNode = getNode(loc);
       if (locNode instanceof InnerNodeWithNodeGroup) {
         InnerNodeWithNodeGroup node = (InnerNodeWithNodeGroup) locNode;
@@ -238,7 +238,7 @@ public class NetworkTopologyWithNodeGroup extends NetworkTopology {
       if (clusterMap.remove(node)) {
         Node nodeGroup = getNode(node.getNetworkLocation());
         if (nodeGroup == null) {
-          nodeGroup = new InnerNode(node.getNetworkLocation());
+          nodeGroup = factory.newInnerNode(node.getNetworkLocation());
         }
         InnerNode rack = (InnerNode)getNode(nodeGroup.getNetworkLocation());
         if (rack == null) {
@@ -302,16 +302,7 @@ public class NetworkTopologyWithNodeGroup extends NetworkTopology {
   /** InnerNodeWithNodeGroup represents a switch/router of a data center, rack
    * or physical host. Different from a leaf node, it has non-null children.
    */
-  static class InnerNodeWithNodeGroup extends InnerNode {
-    public InnerNodeWithNodeGroup(String name, String location, 
-        InnerNode parent, int level) {
-      super(name, location, parent, level);
-    }
-
-    public InnerNodeWithNodeGroup(String name, String location) {
-      super(name, location);
-    }
-
+  static class InnerNodeWithNodeGroup extends InnerNodeImpl {
     public InnerNodeWithNodeGroup(String path) {
       super(path);
     }
@@ -323,10 +314,10 @@ public class NetworkTopologyWithNodeGroup extends NetworkTopology {
         return false;
       }
 
-      Node firstChild = children.get(0);
+      Node firstChild = getChildren().get(0);
 
       if (firstChild instanceof InnerNode) {
-        Node firstGrandChild = (((InnerNode) firstChild).children).get(0);
+        Node firstGrandChild = (((InnerNode) firstChild).getChildren()).get(0);
         if (firstGrandChild instanceof InnerNode) {
           // it is datacenter
           return false;
@@ -343,31 +334,15 @@ public class NetworkTopologyWithNodeGroup extends NetworkTopology {
      * @return true if it has no child or its children are not InnerNodes
      */
     boolean isNodeGroup() {
-      if (children.isEmpty()) {
+      if (getChildren().isEmpty()) {
         return true;
       }
-      Node firstChild = children.get(0);
+      Node firstChild = getChildren().get(0);
       if (firstChild instanceof InnerNode) {
         // it is rack or datacenter
         return false;
       }
       return true;
     }
-    
-    @Override
-    protected boolean isLeafParent() {
-      return isNodeGroup();
-    }
-
-    @Override
-    protected InnerNode createParentNode(String parentName) {
-      return new InnerNodeWithNodeGroup(parentName, getPath(this), this,
-          this.getLevel() + 1);
-    }
-
-    @Override
-    protected boolean areChildrenLeaves() {
-      return isNodeGroup();
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/003ae006/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java
index 3a281fc..62bd262 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java
@@ -296,7 +296,7 @@ public class TestNetworkTopology {
       assertFalse(cluster.contains(dataNodes[i]));
     }
     assertEquals(0, cluster.getNumOfLeaves());
-    assertEquals(0, cluster.clusterMap.children.size());
+    assertEquals(0, cluster.clusterMap.getChildren().size());
     for(int i=0; i<dataNodes.length; i++) {
       cluster.add(dataNodes[i]);
     }


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


[42/50] [abbrv] hadoop git commit: YARN-6159. Documentation changes for TimelineV2Client (Naganarasimha G R via Varun Saxena)

Posted by xg...@apache.org.
YARN-6159. Documentation changes for TimelineV2Client (Naganarasimha G R via Varun Saxena)


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

Branch: refs/heads/YARN-5734
Commit: 6ba61d20d3f65e40ea8e3a49d5beebe34f04aab4
Parents: 8035749
Author: Varun Saxena <va...@apache.org>
Authored: Tue Feb 21 12:25:37 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Tue Feb 21 12:25:37 2017 +0530

----------------------------------------------------------------------
 .../src/site/markdown/TimelineServiceV2.md      | 44 ++++++++------------
 1 file changed, 18 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ba61d20/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
index 90c7a89..dc16803 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
@@ -257,56 +257,48 @@ To write MapReduce framework data to Timeline Service v.2, enable the following
 
 This section is for YARN application developers that want to integrate with Timeline Service v.2.
 
-Developers can continue to use the `TimelineClient` API to publish per-framework data to the
-Timeline Service v.2. You only need to instantiate the right type of the client to write to v.2.
-On the other hand, the entity/object API for v.2 is different than v.1 as the object model is
-significantly changed. The v.2 timeline entity class is
-`org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity` whereas the v.1 class is
-`org.apache.hadoop.yarn.api.records.timeline.TimelineEntity`. The methods on `TimelineClient`
-suitable for writing to Timeline Service v.2 are clearly delineated, and they use the v.2
-types as arguments.
+Developers need to use the `TimelineV2Client` API to publish per-framework data to the
+Timeline Service v.2. The entity/object API for v.2 is different than v.1 as
+the object model is significantly changed. The v.2 timeline entity class is
+`org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity`.
 
 Timeline Service v.2 `putEntities` methods come in 2 varieties: `putEntities` and
 `putEntitiesAsync`. The former is a blocking operation which must be used for writing more
 critical data (e.g. lifecycle events). The latter is a non-blocking operation. Note that neither
 has a return value.
 
-Creating a `TimelineClient` for v.2 involves passing in the application id to the factory method.
+Creating a `TimelineV2Client` involves passing in the application id to the static method
+`TimelineV2Client.createTimelineClient`.
 
 For example:
 
 
     // Create and start the Timeline client v.2
-    TimelineClient client = TimelineClient.createTimelineClient(appId);
-    client.init(conf);
-    client.start();
+    TimelineV2Client timelineClient =
+        TimelineV2Client.createTimelineClient(appId);
+    timelineClient.init(conf);
+    timelineClient.start();
 
     try {
       TimelineEntity myEntity = new TimelineEntity();
-      myEntity.setEntityType("MY_APPLICATION");
-      myEntity.setEntityId("MyApp1")
+      myEntity.setType("MY_APPLICATION");
+      myEntity.setId("MyApp1");
       // Compose other entity info
 
       // Blocking write
-      client.putEntities(entity);
+      timelineClient.putEntities(myEntity);
 
       TimelineEntity myEntity2 = new TimelineEntity();
       // Compose other info
 
       // Non-blocking write
-      timelineClient.putEntitiesAsync(entity);
+      timelineClient.putEntitiesAsync(myEntity2);
 
-    } 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) {
+    } catch (IOException | YarnException e) {
       // Handle the exception
     } finally {
       // Stop the Timeline client
-      client.stop();
+      timelineClient.stop();
     }
 
 As evidenced above, you need to specify the YARN application id to be able to write to the Timeline
@@ -314,9 +306,9 @@ Service v.2. Note that currently you need to be on the cluster to be able to wri
 Service. For example, an application master or code in the container can write to the Timeline
 Service, while an off-cluster MapReduce job submitter cannot.
 
-After creating the timeline client, user also needs to set the timeline collector address for the application. If `AMRMClient` is used then by registering the timeline client by calling `AMRMClient#registerTimelineClient` is sufficient.
+After creating the timeline v2 client, user also needs to set the timeline collector address for the application. If `AMRMClient` is used then by registering the timeline client by calling `AMRMClient#registerTimelineV2Client` is sufficient.
 
-    amRMClient.registerTimelineClient(timelineClient)\u037e
+    amRMClient.registerTimelineV2Client(timelineClient)\u037e
 
 Else address needs to be retrieved from the AM allocate response and need to be set in timeline client explicitly.
 


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


[11/50] [abbrv] hadoop git commit: HDFS-11238. Fix checkstyle warnings in NameNode#createNameNode. Contributed by Ethan Li.

Posted by xg...@apache.org.
HDFS-11238. Fix checkstyle warnings in NameNode#createNameNode. Contributed by Ethan Li.


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

Branch: refs/heads/YARN-5734
Commit: 8acb376c9c5f7f52a097be221ed18877a403bece
Parents: 1e11080
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Feb 15 16:53:50 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Feb 15 16:53:50 2017 +0900

----------------------------------------------------------------------
 .../hadoop/hdfs/server/namenode/NameNode.java   | 101 +++++++++----------
 1 file changed, 46 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8acb376c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index df5ee0f..1752cf7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -1579,62 +1579,53 @@ public class NameNode extends ReconfigurableBase implements
     }
     setStartupOption(conf, startOpt);
 
+    boolean aborted = false;
     switch (startOpt) {
-      case FORMAT: {
-        boolean aborted = format(conf, startOpt.getForceFormat(),
-            startOpt.getInteractiveFormat());
-        terminate(aborted ? 1 : 0);
-        return null; // avoid javac warning
-      }
-      case GENCLUSTERID: {
-        System.err.println("Generating new cluster id:");
-        System.out.println(NNStorage.newClusterID());
-        terminate(0);
-        return null;
-      }
-      case ROLLBACK: {
-        boolean aborted = doRollback(conf, true);
-        terminate(aborted ? 1 : 0);
-        return null; // avoid warning
-      }
-      case BOOTSTRAPSTANDBY: {
-        String toolArgs[] = Arrays.copyOfRange(argv, 1, argv.length);
-        int rc = BootstrapStandby.run(toolArgs, conf);
-        terminate(rc);
-        return null; // avoid warning
-      }
-      case INITIALIZESHAREDEDITS: {
-        boolean aborted = initializeSharedEdits(conf,
-            startOpt.getForceFormat(),
-            startOpt.getInteractiveFormat());
-        terminate(aborted ? 1 : 0);
-        return null; // avoid warning
-      }
-      case BACKUP:
-      case CHECKPOINT: {
-        NamenodeRole role = startOpt.toNodeRole();
-        DefaultMetricsSystem.initialize(role.toString().replace(" ", ""));
-        return new BackupNode(conf, role);
-      }
-      case RECOVER: {
-        NameNode.doRecovery(startOpt, conf);
-        return null;
-      }
-      case METADATAVERSION: {
-        printMetadataVersion(conf);
-        terminate(0);
-        return null; // avoid javac warning
-      }
-      case UPGRADEONLY: {
-        DefaultMetricsSystem.initialize("NameNode");
-        new NameNode(conf);
-        terminate(0);
-        return null;
-      }
-      default: {
-        DefaultMetricsSystem.initialize("NameNode");
-        return new NameNode(conf);
-      }
+    case FORMAT:
+      aborted = format(conf, startOpt.getForceFormat(),
+          startOpt.getInteractiveFormat());
+      terminate(aborted ? 1 : 0);
+      return null; // avoid javac warning
+    case GENCLUSTERID:
+      System.err.println("Generating new cluster id:");
+      System.out.println(NNStorage.newClusterID());
+      terminate(0);
+      return null;
+    case ROLLBACK:
+      aborted = doRollback(conf, true);
+      terminate(aborted ? 1 : 0);
+      return null; // avoid warning
+    case BOOTSTRAPSTANDBY:
+      String[] toolArgs = Arrays.copyOfRange(argv, 1, argv.length);
+      int rc = BootstrapStandby.run(toolArgs, conf);
+      terminate(rc);
+      return null; // avoid warning
+    case INITIALIZESHAREDEDITS:
+      aborted = initializeSharedEdits(conf,
+          startOpt.getForceFormat(),
+          startOpt.getInteractiveFormat());
+      terminate(aborted ? 1 : 0);
+      return null; // avoid warning
+    case BACKUP:
+    case CHECKPOINT:
+      NamenodeRole role = startOpt.toNodeRole();
+      DefaultMetricsSystem.initialize(role.toString().replace(" ", ""));
+      return new BackupNode(conf, role);
+    case RECOVER:
+      NameNode.doRecovery(startOpt, conf);
+      return null;
+    case METADATAVERSION:
+      printMetadataVersion(conf);
+      terminate(0);
+      return null; // avoid javac warning
+    case UPGRADEONLY:
+      DefaultMetricsSystem.initialize("NameNode");
+      new NameNode(conf);
+      terminate(0);
+      return null;
+    default:
+      DefaultMetricsSystem.initialize("NameNode");
+      return new NameNode(conf);
     }
   }
 


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


[34/50] [abbrv] hadoop git commit: HDFS-11410. Use the cached instance when edit logging SetAclOp, SetXAttrOp and RemoveXAttrOp.

Posted by xg...@apache.org.
HDFS-11410. Use the cached instance when edit logging SetAclOp, SetXAttrOp and RemoveXAttrOp.


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

Branch: refs/heads/YARN-5734
Commit: 02c549484a4fe6215c7f1a18d89389dbba6ea723
Parents: 658702e
Author: Xiao Chen <xi...@apache.org>
Authored: Thu Feb 16 18:07:55 2017 -0800
Committer: Xiao Chen <xi...@apache.org>
Committed: Thu Feb 16 18:07:55 2017 -0800

----------------------------------------------------------------------
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |  6 +++---
 .../hdfs/server/namenode/FSEditLogOp.java       | 21 +++++++++-----------
 2 files changed, 12 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/02c54948/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index 8454a46..d3f4447 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -1206,14 +1206,14 @@ public class FSEditLog implements LogsPurgeable {
   }
 
   void logSetAcl(String src, List<AclEntry> entries) {
-    SetAclOp op = SetAclOp.getInstance();
+    final SetAclOp op = SetAclOp.getInstance(cache.get());
     op.src = src;
     op.aclEntries = entries;
     logEdit(op);
   }
   
   void logSetXAttrs(String src, List<XAttr> xAttrs, boolean toLogRpcIds) {
-    final SetXAttrOp op = SetXAttrOp.getInstance();
+    final SetXAttrOp op = SetXAttrOp.getInstance(cache.get());
     op.src = src;
     op.xAttrs = xAttrs;
     logRpcIds(op, toLogRpcIds);
@@ -1221,7 +1221,7 @@ public class FSEditLog implements LogsPurgeable {
   }
   
   void logRemoveXAttrs(String src, List<XAttr> xAttrs, boolean toLogRpcIds) {
-    final RemoveXAttrOp op = RemoveXAttrOp.getInstance();
+    final RemoveXAttrOp op = RemoveXAttrOp.getInstance(cache.get());
     op.src = src;
     op.xAttrs = xAttrs;
     logRpcIds(op, toLogRpcIds);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/02c54948/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
index a3285a9..6293557 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
@@ -3745,8 +3745,7 @@ public abstract class FSEditLogOp {
     }
 
     static AddCacheDirectiveInfoOp getInstance(OpInstanceCache cache) {
-      return (AddCacheDirectiveInfoOp) cache
-          .get(OP_ADD_CACHE_DIRECTIVE);
+      return (AddCacheDirectiveInfoOp) cache.get(OP_ADD_CACHE_DIRECTIVE);
     }
 
     @Override
@@ -3816,8 +3815,7 @@ public abstract class FSEditLogOp {
     }
 
     static ModifyCacheDirectiveInfoOp getInstance(OpInstanceCache cache) {
-      return (ModifyCacheDirectiveInfoOp) cache
-          .get(OP_MODIFY_CACHE_DIRECTIVE);
+      return (ModifyCacheDirectiveInfoOp) cache.get(OP_MODIFY_CACHE_DIRECTIVE);
     }
 
     @Override
@@ -3893,8 +3891,7 @@ public abstract class FSEditLogOp {
     }
 
     static RemoveCacheDirectiveInfoOp getInstance(OpInstanceCache cache) {
-      return (RemoveCacheDirectiveInfoOp) cache
-          .get(OP_REMOVE_CACHE_DIRECTIVE);
+      return (RemoveCacheDirectiveInfoOp) cache.get(OP_REMOVE_CACHE_DIRECTIVE);
     }
 
     @Override
@@ -4146,8 +4143,8 @@ public abstract class FSEditLogOp {
       super(OP_REMOVE_XATTR);
     }
     
-    static RemoveXAttrOp getInstance() {
-      return new RemoveXAttrOp();
+    static RemoveXAttrOp getInstance(OpInstanceCache cache) {
+      return (RemoveXAttrOp) cache.get(OP_REMOVE_XATTR);
     }
 
     @Override
@@ -4199,8 +4196,8 @@ public abstract class FSEditLogOp {
       super(OP_SET_XATTR);
     }
     
-    static SetXAttrOp getInstance() {
-      return new SetXAttrOp();
+    static SetXAttrOp getInstance(OpInstanceCache cache) {
+      return (SetXAttrOp) cache.get(OP_SET_XATTR);
     }
 
     @Override
@@ -4252,8 +4249,8 @@ public abstract class FSEditLogOp {
       super(OP_SET_ACL);
     }
 
-    static SetAclOp getInstance() {
-      return new SetAclOp();
+    static SetAclOp getInstance(OpInstanceCache cache) {
+      return (SetAclOp) cache.get(OP_SET_ACL);
     }
 
     @Override


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


[32/50] [abbrv] hadoop git commit: HADOOP-14019. Fix some typos in the s3a docs. Contributed by Steve Loughran

Posted by xg...@apache.org.
HADOOP-14019. Fix some typos in the s3a docs. 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/bdad8b7b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bdad8b7b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bdad8b7b

Branch: refs/heads/YARN-5734
Commit: bdad8b7b97d7f48119f016d68f32982d680c8796
Parents: f432999
Author: Mingliang Liu <li...@apache.org>
Authored: Thu Feb 16 16:41:31 2017 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Thu Feb 16 16:41:31 2017 -0800

----------------------------------------------------------------------
 .../src/site/markdown/tools/hadoop-aws/index.md     | 16 +++++++++++++---
 1 file changed, 13 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bdad8b7b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index 2471a52..0ff314c 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -970,7 +970,7 @@ This is because the property values are kept in these files, and cannot be
 dynamically patched.
 
 Instead, callers need to create different configuration files for each
-bucket, setting the base secrets (`fs.s3a.bucket.nightly.access.key`, etc),
+bucket, setting the base secrets (`fs.s3a.access.key`, etc),
 then declare the path to the appropriate credential file in
 a bucket-specific version of the property `fs.s3a.security.credential.provider.path`.
 
@@ -1044,7 +1044,7 @@ declaration. For example:
 ### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload
 
 
-**New in Hadoop 2.7; significantly enhanced in Hadoop 2.9**
+**New in Hadoop 2.7; significantly enhanced in Hadoop 2.8**
 
 
 Because of the nature of the S3 object store, data written to an S3A `OutputStream`
@@ -1204,8 +1204,18 @@ consumed, and so eliminates heap size as the limiting factor in queued uploads
   <value>disk</value>
 </property>
 
+<property>
+  <name>fs.s3a.buffer.dir</name>
+  <value></value>
+  <description>Comma separated list of temporary directories use for
+  storing blocks of data prior to their being uploaded to S3.
+  When unset, the Hadoop temporary directory hadoop.tmp.dir is used</description>
+</property>
+
 ```
 
+This is the default buffer mechanism. The amount of data which can
+be buffered is limited by the amount of available disk space.
 
 #### <a name="s3a_fast_upload_bytebuffer"></a>Fast Upload with ByteBuffers: `fs.s3a.fast.upload.buffer=bytebuffer`
 
@@ -1219,7 +1229,7 @@ The amount of data which can be buffered is
 limited by the Java runtime, the operating system, and, for YARN applications,
 the amount of memory requested for each container.
 
-The slower the write bandwidth to S3, the greater the risk of running out
+The slower the upload bandwidth to S3, the greater the risk of running out
 of memory \u2014and so the more care is needed in
 [tuning the upload settings](#s3a_fast_upload_thread_tuning).
 


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


[07/50] [abbrv] hadoop git commit: HDFS-11391. Numeric usernames do no work with WebHDFS FS write access. (Pierre Villard via Yongjun Zhang)

Posted by xg...@apache.org.
HDFS-11391. Numeric usernames do no work with WebHDFS FS write access. (Pierre Villard via Yongjun Zhang)


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

Branch: refs/heads/YARN-5734
Commit: 8e53f2b9b08560bf4f8e81e697063277dbdc68f9
Parents: 652679a
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Tue Feb 14 12:47:06 2017 -0800
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Tue Feb 14 13:40:53 2017 -0800

----------------------------------------------------------------------
 .../hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java       | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8e53f2b9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java
index 095f41d..f8c15fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.permission.FsCreateModes;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.web.JsonUtil;
@@ -55,6 +56,7 @@ import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.resources.GetOpParam;
 import org.apache.hadoop.hdfs.web.resources.PostOpParam;
 import org.apache.hadoop.hdfs.web.resources.PutOpParam;
+import org.apache.hadoop.hdfs.web.resources.UserParam;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
@@ -108,6 +110,10 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> {
     throws IOException {
     this.conf = conf;
     this.confForCreate = confForCreate;
+    /** set user pattern based on configuration file */
+    UserParam.setUserPattern(
+            conf.get(DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_KEY,
+                    DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_DEFAULT));
   }
 
   @Override


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


[02/50] [abbrv] hadoop git commit: HADOOP-14058. Fix NativeS3FileSystemContractBaseTest#testDirWithDifferentMarkersWorks. Contributed by Yiqun Lin.

Posted by xg...@apache.org.
HADOOP-14058. Fix NativeS3FileSystemContractBaseTest#testDirWithDifferentMarkersWorks. Contributed by Yiqun Lin.


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

Branch: refs/heads/YARN-5734
Commit: b9f8491252f5a23a91a1d695d748556a0fd803ae
Parents: aaf106f
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Feb 15 01:45:56 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Feb 15 01:45:56 2017 +0900

----------------------------------------------------------------------
 .../hadoop/fs/s3native/NativeS3FileSystemContractBaseTest.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9f84912/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/NativeS3FileSystemContractBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/NativeS3FileSystemContractBaseTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/NativeS3FileSystemContractBaseTest.java
index ef223ac..261f79b 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/NativeS3FileSystemContractBaseTest.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/NativeS3FileSystemContractBaseTest.java
@@ -85,7 +85,7 @@ public abstract class NativeS3FileSystemContractBaseTest
 
   public void testDirWithDifferentMarkersWorks() throws Exception {
 
-    for (int i = 0; i < 3; i++) {
+    for (int i = 0; i <= 3; i++) {
       String base = "test/hadoop" + i;
       Path path = path("/" + base);
 


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


[47/50] [abbrv] hadoop git commit: HDFS-11405. Rename "erasurecode" CLI subcommand to "ec". Contributed by Manoj Govindassamy.

Posted by xg...@apache.org.
HDFS-11405. Rename "erasurecode" CLI subcommand to "ec". Contributed by Manoj Govindassamy.


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

Branch: refs/heads/YARN-5734
Commit: fc9ad3ce3aa7d28974b0ac3b554089507c124783
Parents: 924def7
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Feb 21 13:55:27 2017 -0800
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Feb 21 13:55:27 2017 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs                | 4 ++--
 .../java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java     | 2 +-
 .../hadoop-hdfs/src/site/markdown/HDFSCommands.md                | 4 ++--
 .../hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md           | 4 ++--
 .../hadoop-hdfs/src/test/resources/testErasureCodingConf.xml     | 2 +-
 5 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc9ad3ce/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
index f095e9b..617adbe 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -41,7 +41,7 @@ function hadoop_usage
   hadoop_add_subcommand "dfsadmin" "run a DFS admin client"
   hadoop_add_subcommand "diskbalancer" "Distributes data evenly among disks on a given node"
   hadoop_add_subcommand "envvars" "display computed Hadoop environment variables"
-  hadoop_add_subcommand "erasurecode" "run a HDFS ErasureCoding CLI"
+  hadoop_add_subcommand "ec" "run a HDFS ErasureCoding CLI"
   hadoop_add_subcommand "fetchdt" "fetch a delegation token from the NameNode"
   hadoop_add_subcommand "fsck" "run a DFS filesystem checking utility"
   hadoop_add_subcommand "getconf" "get config values from configuration"
@@ -129,7 +129,7 @@ function hdfscmd_case
       echo "HADOOP_TOOLS_LIB_JARS_DIR='${HADOOP_TOOLS_LIB_JARS_DIR}'"
       exit 0
     ;;
-    erasurecode)
+    ec)
       HADOOP_CLASSNAME=org.apache.hadoop.hdfs.tools.erasurecode.ECCli
     ;;
     fetchdt)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc9ad3ce/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java
index 48574d3..89dd4ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java
@@ -32,7 +32,7 @@ import java.io.IOException;
 public class ECCli extends FsShell {
 
   private final static String usagePrefix =
-      "Usage: hdfs erasurecode [generic options]";
+      "Usage: hdfs ec [generic options]";
 
   @Override
   protected String getUsagePrefix() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc9ad3ce/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index 1a4465c..b65cc78 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -431,11 +431,11 @@ Usage:
 
 Runs the diskbalancer CLI. See [HDFS Diskbalancer](./HDFSDiskbalancer.html) for more information on this command.
 
-### `erasurecode`
+### `ec`
 
 Usage:
 
-       hdfs erasurecode [generic options]
+       hdfs ec [generic options]
          [-setPolicy [-p <policyName>] <path>]
          [-getPolicy <path>]
          [-listPolicies]

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc9ad3ce/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
index 8c504ac..6e4891f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
@@ -116,9 +116,9 @@ Deployment
 
 ### Administrative commands
 
-  HDFS provides an `erasurecode` subcommand to perform administrative commands related to erasure coding.
+  HDFS provides an `ec` subcommand to perform administrative commands related to erasure coding.
 
-       hdfs erasurecode [generic options]
+       hdfs ec [generic options]
          [-setPolicy [-p <policyName>] <path>]
          [-getPolicy <path>]
          [-unsetPolicy <path>]

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc9ad3ce/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
index dd26b48..0753d4d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
@@ -58,7 +58,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Usage: hdfs erasurecode [generic options]</expected-output>
+          <expected-output>Usage: hdfs ec [generic options]</expected-output>
         </comparator>
       </comparators>
     </test>


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


[04/50] [abbrv] hadoop git commit: HDFS-11084. Add a regression test for sticky bit support of OIV ReverseXML processor. Contributed by Wei-Chiu Chuang.

Posted by xg...@apache.org.
HDFS-11084. Add a regression test for sticky bit support of OIV ReverseXML processor. Contributed by Wei-Chiu Chuang.


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

Branch: refs/heads/YARN-5734
Commit: 0cf5993712a01993bd701bd9664e6af284378b55
Parents: 1fa084c
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Tue Feb 14 08:59:12 2017 -0800
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Tue Feb 14 09:11:55 2017 -0800

----------------------------------------------------------------------
 .../tools/offlineImageViewer/TestOfflineImageViewer.java | 11 +++++++++++
 1 file changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0cf59937/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
index 740a8ab..dacbb85 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
@@ -69,6 +69,8 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -158,6 +160,15 @@ public class TestOfflineImageViewer {
       hdfs.mkdirs(invalidXMLDir);
       dirCount++;
 
+      //Create a directory with sticky bits
+      Path stickyBitDir = new Path("/stickyBit");
+      hdfs.mkdirs(stickyBitDir);
+      hdfs.setPermission(stickyBitDir, new FsPermission(FsAction.ALL,
+          FsAction.ALL, FsAction.ALL, true));
+      dirCount++;
+      writtenFiles.put(stickyBitDir.toString(),
+          hdfs.getFileStatus(stickyBitDir));
+
       // Get delegation tokens so we log the delegation token op
       Token<?>[] delegationTokens = hdfs
           .addDelegationTokens(TEST_RENEWER, null);


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


[48/50] [abbrv] hadoop git commit: HDFS-11406. Remove unused getStartInstance and getFinalizeInstance in FSEditLogOp. Contributed by Alison Yu.

Posted by xg...@apache.org.
HDFS-11406. Remove unused getStartInstance and getFinalizeInstance in FSEditLogOp. Contributed by Alison Yu.


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

Branch: refs/heads/YARN-5734
Commit: 8ef7ebbc7112e1868c9b12ff1df4a40fe7afa8af
Parents: fc9ad3c
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Feb 21 14:54:20 2017 -0800
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Feb 21 14:54:20 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java  | 8 --------
 1 file changed, 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef7ebbc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
index 6293557..f93e867 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
@@ -4351,14 +4351,6 @@ public abstract class FSEditLogOp {
       this.name = StringUtils.toUpperCase(name);
     }
 
-    static RollingUpgradeOp getStartInstance(OpInstanceCache cache) {
-      return (RollingUpgradeOp) cache.get(OP_ROLLING_UPGRADE_START);
-    }
-
-    static RollingUpgradeOp getFinalizeInstance(OpInstanceCache cache) {
-      return (RollingUpgradeOp) cache.get(OP_ROLLING_UPGRADE_FINALIZE);
-    }
-
     @Override
     void resetSubFields() {
       time = 0L;


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


[05/50] [abbrv] hadoop git commit: HDFS-11409. DatanodeInfo getNetworkLocation and setNetworkLocation shoud use volatile instead of synchronized. Contributed by Chen Liang.

Posted by xg...@apache.org.
HDFS-11409. DatanodeInfo getNetworkLocation and setNetworkLocation shoud use volatile instead of synchronized. Contributed by Chen Liang.


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

Branch: refs/heads/YARN-5734
Commit: aaf27132350547fcde1fdb372f19626838f44bc4
Parents: 0cf5993
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Tue Feb 14 12:52:34 2017 -0800
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Tue Feb 14 12:52:34 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java     | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aaf27132/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
index 41735b1..acbcffa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
@@ -51,7 +51,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
   private long lastUpdate;
   private long lastUpdateMonotonic;
   private int xceiverCount;
-  private String location = NetworkTopology.DEFAULT_RACK;
+  private volatile String location = NetworkTopology.DEFAULT_RACK;
   private String softwareVersion;
   private List<String> dependentHostNames = new LinkedList<>();
   private String upgradeDomain;
@@ -293,11 +293,11 @@ public class DatanodeInfo extends DatanodeID implements Node {
 
   /** network location */
   @Override
-  public synchronized String getNetworkLocation() {return location;}
+  public String getNetworkLocation() {return location;}
 
   /** Sets the network location */
   @Override
-  public synchronized void setNetworkLocation(String location) {
+  public void setNetworkLocation(String location) {
     this.location = NodeBase.normalize(location);
   }
 


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


[26/50] [abbrv] hadoop git commit: HDFS-11100. Recursively deleting file protected by sticky bit should fail. Contributed by John Zhuge.

Posted by xg...@apache.org.
HDFS-11100. Recursively deleting file protected by sticky bit should fail. Contributed by John Zhuge.


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

Branch: refs/heads/YARN-5734
Commit: 5690b51ef7c708c0a71162ddaff04466bc71cdcc
Parents: e63a781
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Thu Feb 16 05:39:37 2017 -0800
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Thu Feb 16 05:39:37 2017 -0800

----------------------------------------------------------------------
 .../apache/hadoop/fs/FSExceptionMessages.java   |  3 +
 .../server/namenode/FSPermissionChecker.java    | 87 +++++++++++++++++---
 .../hadoop/fs/permission/TestStickyBit.java     | 63 ++++++++++++++
 3 files changed, 142 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5690b51e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSExceptionMessages.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSExceptionMessages.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSExceptionMessages.java
index 1511bb0..a8e7b71 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSExceptionMessages.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSExceptionMessages.java
@@ -48,4 +48,7 @@ public class FSExceptionMessages {
       = "Requested more bytes than destination buffer size";
 
   public static final String PERMISSION_DENIED = "Permission denied";
+
+  public static final String PERMISSION_DENIED_BY_STICKY_BIT =
+      "Permission denied by sticky bit";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5690b51e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
index 107d563..f1250dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
@@ -18,11 +18,15 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
 import java.util.Stack;
 
+import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.permission.AclEntryScope;
 import org.apache.hadoop.fs.permission.AclEntryType;
@@ -280,9 +284,20 @@ class FSPermissionChecker implements AccessControlEnforcer {
       return;
     }
 
+    // Each inode in the subtree has a level. The root inode has level 0.
+    // List subINodePath tracks the inode path in the subtree during
+    // traversal. The root inode is not stored because it is already in array
+    // components. The list index is (level - 1).
+    ArrayList<INodeDirectory> subINodePath = new ArrayList<>();
+
+    // The stack of levels matches the stack of directory inodes.
+    Stack<Integer> levels = new Stack<>();
+    levels.push(0);    // Level 0 is the root
+
     Stack<INodeDirectory> directories = new Stack<INodeDirectory>();
     for(directories.push(inode.asDirectory()); !directories.isEmpty(); ) {
       INodeDirectory d = directories.pop();
+      int level = levels.pop();
       ReadOnlyList<INode> cList = d.getChildrenList(snapshotId);
       if (!(cList.isEmpty() && ignoreEmptyDir)) {
         //TODO have to figure this out with inodeattribute provider
@@ -292,11 +307,44 @@ class FSPermissionChecker implements AccessControlEnforcer {
           throw new AccessControlException(
               toAccessControlString(inodeAttr, d.getFullPathName(), access));
         }
+
+        if (level > 0) {
+          if (level - 1 < subINodePath.size()) {
+            subINodePath.set(level - 1, d);
+          } else {
+            Preconditions.checkState(level - 1 == subINodePath.size());
+            subINodePath.add(d);
+          }
+        }
+
+        if (inodeAttr.getFsPermission().getStickyBit()) {
+          for (INode child : cList) {
+            INodeAttributes childInodeAttr =
+                getINodeAttrs(components, pathIdx, child, snapshotId);
+            if (isStickyBitViolated(inodeAttr, childInodeAttr)) {
+              List<byte[]> allComponentList = new ArrayList<>();
+              for (int i = 0; i <= pathIdx; ++i) {
+                allComponentList.add(components[i]);
+              }
+              for (int i = 0; i < level; ++i) {
+                allComponentList.add(subINodePath.get(i).getLocalNameBytes());
+              }
+              allComponentList.add(child.getLocalNameBytes());
+              int index = pathIdx + level;
+              byte[][] allComponents =
+                  allComponentList.toArray(new byte[][]{});
+              throwStickyBitException(
+                  getPath(allComponents, 0, index + 1), child,
+                  getPath(allComponents, 0, index), inode);
+            }
+          }
+        }
       }
 
       for(INode child : cList) {
         if (child.isDirectory()) {
           directories.push(child.asDirectory());
+          levels.push(level + 1);
         }
       }
     }
@@ -425,26 +473,43 @@ class FSPermissionChecker implements AccessControlEnforcer {
       return;
     }
 
+    INodeAttributes inode = inodes[index + 1];
+    if (!isStickyBitViolated(parent, inode)) {
+      return;
+    }
+
+    throwStickyBitException(getPath(components, 0, index + 1), inode,
+        getPath(components, 0, index), parent);
+  }
+
+  /** Return true when sticky bit is violated. */
+  private boolean isStickyBitViolated(INodeAttributes parent,
+                                      INodeAttributes inode) {
     // If this user is the directory owner, return
     if (parent.getUserName().equals(getUser())) {
-      return;
+      return false;
     }
 
-    INodeAttributes inode = inodes[index + 1];
     // if this user is the file owner, return
     if (inode.getUserName().equals(getUser())) {
-      return;
+      return false;
     }
 
+    return true;
+  }
+
+  private void throwStickyBitException(
+      String inodePath, INodeAttributes inode,
+      String parentPath, INodeAttributes parent)
+      throws AccessControlException {
     throw new AccessControlException(String.format(
-        "Permission denied by sticky bit: user=%s, path=\"%s\":%s:%s:%s%s, " +
-        "parent=\"%s\":%s:%s:%s%s", user,
-        getPath(components, 0, index + 1),
-        inode.getUserName(), inode.getGroupName(),
-        inode.isDirectory() ? "d" : "-", inode.getFsPermission().toString(),
-        getPath(components, 0, index),
-        parent.getUserName(), parent.getGroupName(),
-        parent.isDirectory() ? "d" : "-", parent.getFsPermission().toString()));
+        FSExceptionMessages.PERMISSION_DENIED_BY_STICKY_BIT +
+            ": user=%s, path=\"%s\":%s:%s:%s%s, " +
+            "parent=\"%s\":%s:%s:%s%s", user, inodePath, inode.getUserName(),
+        inode.getGroupName(), inode.isDirectory() ? "d" : "-",
+        inode.getFsPermission().toString(), parentPath, parent.getUserName(),
+        parent.getGroupName(), parent.isDirectory() ? "d" : "-",
+        parent.getFsPermission().toString()));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5690b51e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java
index 99f1b1b..a6409fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/permission/TestStickyBit.java
@@ -32,6 +32,7 @@ import java.util.Arrays;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -43,6 +44,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -426,6 +428,67 @@ public class TestStickyBit {
     assertFalse(hdfs.getFileStatus(sbSetOff).getPermission().getStickyBit());
   }
 
+  @Test
+  public void testStickyBitRecursiveDeleteFile() throws Exception {
+    Path root = new Path("/" + GenericTestUtils.getMethodName());
+    Path tmp = new Path(root, "tmp");
+    Path file = new Path(tmp, "file");
+
+    // Create a tmp directory with wide-open permissions and sticky bit
+    hdfs.mkdirs(tmp);
+    hdfs.setPermission(root, new FsPermission((short) 0777));
+    hdfs.setPermission(tmp, new FsPermission((short) 01777));
+
+    // Create a file protected by sticky bit
+    writeFile(hdfsAsUser1, file);
+    hdfs.setPermission(file, new FsPermission((short) 0666));
+
+    try {
+      hdfsAsUser2.delete(tmp, true);
+      fail("Non-owner can not delete a file protected by sticky bit"
+          + " recursively");
+    } catch (AccessControlException e) {
+      GenericTestUtils.assertExceptionContains(
+          FSExceptionMessages.PERMISSION_DENIED_BY_STICKY_BIT, e);
+    }
+
+    // Owner can delete a file protected by sticky bit recursively
+    hdfsAsUser1.delete(tmp, true);
+  }
+
+  @Test
+  public void testStickyBitRecursiveDeleteDir() throws Exception {
+    Path root = new Path("/" + GenericTestUtils.getMethodName());
+    Path tmp = new Path(root, "tmp");
+    Path dir = new Path(tmp, "dir");
+    Path file = new Path(dir, "file");
+
+    // Create a tmp directory with wide-open permissions and sticky bit
+    hdfs.mkdirs(tmp);
+    hdfs.setPermission(root, new FsPermission((short) 0777));
+    hdfs.setPermission(tmp, new FsPermission((short) 01777));
+
+    // Create a dir protected by sticky bit
+    hdfsAsUser1.mkdirs(dir);
+    hdfsAsUser1.setPermission(dir, new FsPermission((short) 0777));
+
+    // Create a file in dir
+    writeFile(hdfsAsUser1, file);
+    hdfs.setPermission(file, new FsPermission((short) 0666));
+
+    try {
+      hdfsAsUser2.delete(tmp, true);
+      fail("Non-owner can not delete a directory protected by sticky bit"
+          + " recursively");
+    } catch (AccessControlException e) {
+      GenericTestUtils.assertExceptionContains(
+          FSExceptionMessages.PERMISSION_DENIED_BY_STICKY_BIT, e);
+    }
+
+    // Owner can delete a directory protected by sticky bit recursively
+    hdfsAsUser1.delete(tmp, true);
+  }
+
   /***
    * Write a quick file to the specified file system at specified path
    */


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


[03/50] [abbrv] hadoop git commit: YARN-5912. Fix breadcrumb issues for various pages in new YARN UI. Contributed by Akhil P B.

Posted by xg...@apache.org.
YARN-5912. Fix breadcrumb issues for various pages in new YARN UI. Contributed by Akhil P B.


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

Branch: refs/heads/YARN-5734
Commit: 1fa084c4254b89cd45210727ccb68725d583ff62
Parents: b9f8491
Author: Sunil G <su...@apache.org>
Authored: Tue Feb 14 22:29:21 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Tue Feb 14 22:29:21 2017 +0530

----------------------------------------------------------------------
 .../webapp/app/controllers/yarn-app-attempt.js  |  2 +-
 .../webapp/app/controllers/yarn-app-attempts.js |  2 +-
 .../src/main/webapp/app/controllers/yarn-app.js |  2 +-
 .../main/webapp/app/controllers/yarn-apps.js    |  2 +-
 .../app/controllers/yarn-container-log.js       |  7 +++-
 .../webapp/app/controllers/yarn-node-app.js     |  7 +++-
 .../webapp/app/controllers/yarn-node-apps.js    |  2 +-
 .../app/controllers/yarn-node-container.js      | 39 ++++++++++++++++++++
 .../app/controllers/yarn-node-containers.js     |  2 +-
 .../main/webapp/app/controllers/yarn-node.js    |  2 +-
 .../webapp/app/controllers/yarn-services.js     |  2 +-
 .../src/main/webapp/app/models/yarn-app.js      |  2 +-
 .../src/main/webapp/app/routes/yarn-node-app.js |  2 +-
 .../webapp/app/routes/yarn-node-container.js    |  2 +-
 .../controllers/yarn-node-container-test.js     | 30 +++++++++++++++
 15 files changed, 90 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fa084c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app-attempt.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app-attempt.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app-attempt.js
index a458842..4c02361 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app-attempt.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app-attempt.js
@@ -27,7 +27,7 @@ export default Ember.Controller.extend({
       routeName: 'application'
     },{
       text: "Applications",
-      routeName: 'yarn-apps'
+      routeName: 'yarn-apps.apps'
     }, {
       text: `App [${appId}]`,
       routeName: 'yarn-app',

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fa084c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app-attempts.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app-attempts.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app-attempts.js
index 9ebc2a6..92de2f9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app-attempts.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app-attempts.js
@@ -27,7 +27,7 @@ export default Ember.Controller.extend({
       routeName: 'application'
     },{
       text: "Applications",
-      routeName: 'yarn-apps'
+      routeName: 'yarn-apps.apps'
     }, {
       text: `App [${appId}]`,
       routeName: 'yarn-app',

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fa084c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app.js
index 309c895..9c1cb5d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app.js
@@ -27,7 +27,7 @@ export default Ember.Controller.extend({
       routeName: 'application'
     },{
       text: "Applications",
-      routeName: 'yarn-apps'
+      routeName: 'yarn-apps.apps'
     }, {
       text: `App [${appId}]`,
       routeName: 'yarn-app',

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fa084c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-apps.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-apps.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-apps.js
index 396f83b..18bf682 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-apps.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-apps.js
@@ -25,7 +25,7 @@ export default Ember.Controller.extend({
     routeName: 'application'
   }, {
     text: "Applications",
-    routeName: 'yarn-apps',
+    routeName: 'yarn-apps.apps',
   }]
 
 });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fa084c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-container-log.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-container-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-container-log.js
index 3352eaf..db7470d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-container-log.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-container-log.js
@@ -27,11 +27,14 @@ export default Ember.Controller.extend({
       text: "Home",
       routeName: 'application'
     }, {
+      text: "Nodes",
+      routeName: 'yarn-nodes.table'
+    }, {
       text: `Node [ ${nodeInfo.id} ]`,
-      href: `/#/yarn-node/${nodeInfo.id}/${nodeInfo.addr}`,
+      href: `#/yarn-node/${nodeInfo.id}/${nodeInfo.addr}`,
     }, {
       text: `Container [ ${containerInfo.id} ]`,
-      href: `/#/yarn-node-container/${nodeInfo.id}/${nodeInfo.addr}/${containerInfo.id}`,
+      href: `#/yarn-node-container/${nodeInfo.id}/${nodeInfo.addr}/${containerInfo.id}`,
     }, {
       text: "Log",
     }];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fa084c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-app.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-app.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-app.js
index 76da281..3dc09fc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-app.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-app.js
@@ -26,10 +26,13 @@ export default Ember.Controller.extend({
       text: "Home",
       routeName: 'application'
     }, {
+      text: "Nodes",
+      routeName: 'yarn-nodes.table'
+    }, {
       text: `Node [ ${nodeInfo.id} ]`,
-      href: `/#/yarn-node/${nodeInfo.id}/${nodeInfo.addr}`,
+      href: `#/yarn-node/${nodeInfo.id}/${nodeInfo.addr}`,
     }, {
-      text: "Application",
+      text: `Application [ ${nodeInfo.appId} ]`,
     }];
   })
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fa084c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-apps.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-apps.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-apps.js
index 6e67ab0..6f53e74 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-apps.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-apps.js
@@ -30,7 +30,7 @@ export default Ember.Controller.extend({
       routeName: 'yarn-nodes.table'
     }, {
       text: `Node [ ${nodeInfo.id} ]`,
-      href: `/#/yarn-node/${nodeInfo.id}/${nodeInfo.addr}`,
+      href: `#/yarn-node/${nodeInfo.id}/${nodeInfo.addr}`
     }, {
       text: "Applications",
     }];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fa084c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-container.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-container.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-container.js
new file mode 100644
index 0000000..afcd518
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-container.js
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Ember from 'ember';
+
+export default Ember.Controller.extend({
+
+  breadcrumbs: Ember.computed("model.nodeInfo", function () {
+    var nodeInfo = this.get("model.nodeInfo");
+    return [{
+      text: "Home",
+      routeName: 'application'
+    },{
+      text: "Nodes",
+      routeName: 'yarn-nodes.table'
+    }, {
+      text: `Node [ ${nodeInfo.id} ]`,
+      href: `#/yarn-node/${nodeInfo.id}/${nodeInfo.addr}`
+    }, {
+      text: `Container [ ${nodeInfo.containerId} ]`
+    }];
+  })
+
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fa084c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-containers.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-containers.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-containers.js
index abe4098..21d50a3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-containers.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-containers.js
@@ -30,7 +30,7 @@ export default Ember.Controller.extend({
       routeName: 'yarn-nodes.table'
     }, {
       text: `Node [ ${nodeInfo.id} ]`,
-      href: `/#/yarn-node/${nodeInfo.id}/${nodeInfo.addr}`,
+      href: `#/yarn-node/${nodeInfo.id}/${nodeInfo.addr}`
     }, {
       text: "Containers",
     }];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fa084c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node.js
index 0661415..55d9b3e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node.js
@@ -30,7 +30,7 @@ export default Ember.Controller.extend({
       routeName: 'yarn-nodes.table'
     }, {
       text: `Node [ ${nodeInfo.id} ]`,
-      href: `/#/yarn-node/${nodeInfo.id}/${nodeInfo.addr}`,
+      href: `#/yarn-node/${nodeInfo.id}/${nodeInfo.addr}`,
     }];
   })
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fa084c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-services.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-services.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-services.js
index 597962a..75bebf6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-services.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-services.js
@@ -25,7 +25,7 @@ export default Ember.Controller.extend({
     routeName: 'application'
   }, {
     text: "Applications",
-    routeName: 'yarn-apps',
+    routeName: 'yarn-apps.apps',
   }, {
     text: "Long Running Services",
     routeName: 'yarn-services',

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fa084c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app.js
index ea1334f..1366357 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-app.js
@@ -54,7 +54,7 @@ export default DS.Model.extend({
   applicationExpiryTime: DS.attr('string'),
 
   isFailed: function() {
-    return this.get('finalStatus') == "FAILED";
+    return this.get('finalStatus') === "FAILED";
   }.property("finalStatus"),
 
   validatedFinishedTs: function() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fa084c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-node-app.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-node-app.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-node-app.js
index 0a11930..8b2ad80 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-node-app.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-node-app.js
@@ -25,7 +25,7 @@ export default AbstractRoute.extend({
     return Ember.RSVP.hash({
       nodeApp: this.store.queryRecord('yarn-node-app',
           { nodeAddr : param.node_addr, appId: param.app_id }),
-      nodeInfo: { id: param.node_id, addr: param.node_addr }
+      nodeInfo: { id: param.node_id, addr: param.node_addr, appId: param.app_id }
     });
   },
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fa084c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-node-container.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-node-container.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-node-container.js
index b7a79de..5708fea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-node-container.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-node-container.js
@@ -26,7 +26,7 @@ export default AbstractRoute.extend({
     return Ember.RSVP.hash({
       nodeContainer: this.store.queryRecord('yarn-node-container',
           { nodeHttpAddr: param.node_addr, containerId: param.container_id }),
-      nodeInfo: { id: param.node_id, addr: param.node_addr }
+      nodeInfo: { id: param.node_id, addr: param.node_addr, containerId: param.container_id }
     });
   },
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1fa084c4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-node-container-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-node-container-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-node-container-test.js
new file mode 100644
index 0000000..c35e413
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-node-container-test.js
@@ -0,0 +1,30 @@
+/**
+ * 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.
+ */
+
+import { moduleFor, test } from 'ember-qunit';
+
+moduleFor('controller:yarn-node-container', 'Unit | Controller | yarn node container', {
+  // Specify the other units that are required for this test.
+  // needs: ['controller:foo']
+});
+
+// Replace this with your real tests.
+test('it exists', function(assert) {
+  let controller = this.subject();
+  assert.ok(controller);
+});


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


[46/50] [abbrv] hadoop git commit: HADOOP-14017. User friendly name for ADLS user and group. Contributed by Vishwajeet Dusane

Posted by xg...@apache.org.
HADOOP-14017. User friendly name for ADLS user and group. Contributed by Vishwajeet Dusane


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

Branch: refs/heads/YARN-5734
Commit: 924def78544a64449785f305cb6984c3559aea4d
Parents: 2158496
Author: Mingliang Liu <li...@apache.org>
Authored: Tue Feb 21 13:44:42 2017 -0800
Committer: Mingliang Liu <li...@apache.org>
Committed: Tue Feb 21 13:44:42 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/adl/AdlConfKeys.java   |  4 +++
 .../org/apache/hadoop/fs/adl/AdlFileSystem.java | 23 +++++++++++---
 .../src/site/markdown/index.md                  | 26 +++++++++++++++
 .../fs/adl/TestValidateConfiguration.java       |  9 ++++++
 .../apache/hadoop/fs/adl/live/TestMetadata.java | 33 ++++++++++++++++++++
 5 files changed, 91 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/924def78/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java
index 21120df..7d31103 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java
@@ -87,6 +87,10 @@ public final class AdlConfKeys {
       "adl.feature.support.acl.bit";
   static final boolean ADL_SUPPORT_ACL_BIT_IN_FSPERMISSION_DEFAULT = true;
 
+  static final String ADL_ENABLEUPN_FOR_OWNERGROUP_KEY =
+      "adl.feature.ownerandgroup.enableupn";
+  static final boolean ADL_ENABLEUPN_FOR_OWNERGROUP_DEFAULT = false;
+
   private AdlConfKeys() {
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/924def78/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
index fb0feda..e0e273e 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
@@ -32,6 +32,7 @@ import com.microsoft.azure.datalake.store.DirectoryEntry;
 import com.microsoft.azure.datalake.store.DirectoryEntryType;
 import com.microsoft.azure.datalake.store.IfExists;
 import com.microsoft.azure.datalake.store.LatencyTracker;
+import com.microsoft.azure.datalake.store.UserGroupRepresentation;
 import com.microsoft.azure.datalake.store.oauth2.AccessTokenProvider;
 import com.microsoft.azure.datalake.store.oauth2.ClientCredsTokenProvider;
 import com.microsoft.azure.datalake.store.oauth2.RefreshTokenBasedTokenProvider;
@@ -80,6 +81,8 @@ public class AdlFileSystem extends FileSystem {
   private ADLStoreClient adlClient;
   private Path workingDirectory;
   private boolean aclBitStatus;
+  private UserGroupRepresentation oidOrUpn;
+
 
   // retained for tests
   private AccessTokenProvider tokenProvider;
@@ -181,6 +184,11 @@ public class AdlFileSystem extends FileSystem {
     if (!trackLatency) {
       LatencyTracker.disable();
     }
+
+    boolean enableUPN = conf.getBoolean(ADL_ENABLEUPN_FOR_OWNERGROUP_KEY,
+        ADL_ENABLEUPN_FOR_OWNERGROUP_DEFAULT);
+    oidOrUpn = enableUPN ? UserGroupRepresentation.UPN :
+        UserGroupRepresentation.OID;
   }
 
   /**
@@ -439,7 +447,8 @@ public class AdlFileSystem extends FileSystem {
   @Override
   public FileStatus getFileStatus(final Path f) throws IOException {
     statistics.incrementReadOps(1);
-    DirectoryEntry entry = adlClient.getDirectoryEntry(toRelativeFilePath(f));
+    DirectoryEntry entry =
+        adlClient.getDirectoryEntry(toRelativeFilePath(f), oidOrUpn);
     return toFileStatus(entry, f);
   }
 
@@ -456,7 +465,7 @@ public class AdlFileSystem extends FileSystem {
   public FileStatus[] listStatus(final Path f) throws IOException {
     statistics.incrementReadOps(1);
     List<DirectoryEntry> entries =
-        adlClient.enumerateDirectory(toRelativeFilePath(f));
+        adlClient.enumerateDirectory(toRelativeFilePath(f), oidOrUpn);
     return toFileStatuses(entries, f);
   }
 
@@ -749,8 +758,8 @@ public class AdlFileSystem extends FileSystem {
   @Override
   public AclStatus getAclStatus(final Path path) throws IOException {
     statistics.incrementReadOps(1);
-    com.microsoft.azure.datalake.store.acl.AclStatus adlStatus = adlClient
-        .getAclStatus(toRelativeFilePath(path));
+    com.microsoft.azure.datalake.store.acl.AclStatus adlStatus =
+        adlClient.getAclStatus(toRelativeFilePath(path), oidOrUpn);
     AclStatus.Builder aclStatusBuilder = new AclStatus.Builder();
     aclStatusBuilder.owner(adlStatus.owner);
     aclStatusBuilder.group(adlStatus.group);
@@ -963,4 +972,10 @@ public class AdlFileSystem extends FileSystem {
     }
     return new String(passchars);
   }
+
+  @VisibleForTesting
+  public void setUserGroupRepresentationAsUPN(boolean enableUPN) {
+    oidOrUpn = enableUPN ? UserGroupRepresentation.UPN :
+        UserGroupRepresentation.OID;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/924def78/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md b/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
index 5037db6..6d9e173 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
+++ b/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
@@ -26,6 +26,7 @@
         * [Protecting the Credentials with Credential Providers](#Credential_Provider)
     * [Enabling ADL Filesystem](#Enabling_ADL)
     * [Accessing adl URLs](#Accessing_adl_URLs)
+    * [User/Group Representation](#OIDtoUPNConfiguration)
 * [Testing the hadoop-azure Module](#Testing_the_hadoop-azure_Module)
 
 ## <a name="Introduction" />Introduction
@@ -42,6 +43,8 @@ The jar file is named azure-datalake-store.jar.
 * Can act as a source of data in a MapReduce job, or a sink.
 * Tested on both Linux and Windows.
 * Tested for scale.
+* API setOwner/setAcl/removeAclEntries/modifyAclEntries accepts UPN or OID
+  (Object ID) as user and group name.
 
 ## <a name="Limitations" />Limitations
 Partial or no support for the following operations :
@@ -221,6 +224,29 @@ commands demonstrate access to a storage account named `youraccount`.
 
     > hadoop fs -cat adl://yourcontainer.azuredatalakestore.net/testDir/testFile
     test file content
+
+### <a name="OIDtoUPNConfiguration" />User/Group Representation
+The hadoop-azure-datalake module provides support for configuring how
+User/Group information is represented during
+getFileStatus/listStatus/getAclStatus.
+
+Add the following properties to your core-site.xml
+
+        <property>
+          <name>adl.feature.ownerandgroup.enableupn</name>
+          <value>true</value>
+          <description>
+            When true : User and Group in FileStatus/AclStatus response is
+            represented as user friendly name as per Azure AD profile.
+
+            When false (default) : User and Group in FileStatus/AclStatus
+            response is represented by the unique identifier from Azure AD
+            profile (Object ID as GUID).
+
+            For performance optimization, Recommended default value.
+          </description>
+        </property>
+
 ## <a name="Testing_the_hadoop-azure_Module" />Testing the azure-datalake-store Module
 The hadoop-azure module includes a full suite of unit tests. Most of the tests will run without additional configuration by running mvn test. This includes tests against mocked storage, which is an in-memory emulation of Azure Data Lake Storage.
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/924def78/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestValidateConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestValidateConfiguration.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestValidateConfiguration.java
index e3025b2..4cabaa3 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestValidateConfiguration.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestValidateConfiguration.java
@@ -27,6 +27,10 @@ import static org.apache.hadoop.fs.adl.AdlConfKeys
 import static org.apache.hadoop.fs.adl.AdlConfKeys
     .ADL_DEBUG_SET_LOCAL_USER_AS_OWNER_DEFAULT;
 import static org.apache.hadoop.fs.adl.AdlConfKeys
+    .ADL_ENABLEUPN_FOR_OWNERGROUP_DEFAULT;
+import static org.apache.hadoop.fs.adl.AdlConfKeys
+    .ADL_ENABLEUPN_FOR_OWNERGROUP_KEY;
+import static org.apache.hadoop.fs.adl.AdlConfKeys
     .ADL_EXPERIMENT_POSITIONAL_READ_DEFAULT;
 import static org.apache.hadoop.fs.adl.AdlConfKeys
     .ADL_EXPERIMENT_POSITIONAL_READ_KEY;
@@ -99,5 +103,10 @@ public class TestValidateConfiguration {
     Assert.assertEquals(false, ADL_DEBUG_SET_LOCAL_USER_AS_OWNER_DEFAULT);
     Assert.assertEquals(4 * 1024 * 1024, DEFAULT_READ_AHEAD_BUFFER_SIZE);
     Assert.assertEquals(4 * 1024 * 1024, DEFAULT_WRITE_AHEAD_BUFFER_SIZE);
+
+    Assert.assertEquals("adl.feature.ownerandgroup.enableupn",
+        ADL_ENABLEUPN_FOR_OWNERGROUP_KEY);
+    Assert.assertEquals(false,
+        ADL_ENABLEUPN_FOR_OWNERGROUP_DEFAULT);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/924def78/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestMetadata.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestMetadata.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestMetadata.java
index 3b9e7da..dbcaa39 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestMetadata.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/live/TestMetadata.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.adl.AdlFileSystem;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assume;
@@ -32,6 +33,8 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.util.UUID;
 
+import static org.junit.Assert.fail;
+
 /**
  * This class is responsible for testing ContentSummary, ListStatus on
  * file/folder.
@@ -107,5 +110,35 @@ public class TestMetadata {
         .assertEquals(path.makeQualified(fs.getUri(), fs.getWorkingDirectory()),
             statuses[0].getPath());
   }
+
+  @Test
+  public void testUserRepresentationConfiguration() throws IOException {
+    // Validating actual user/group OID or friendly name is outside scope of
+    // this test.
+    Path path = new Path(parent, "a.txt");
+    AdlFileSystem fs = (AdlFileSystem) adlStore;
+
+    // When set to true, User/Group information should be user friendly name.
+    // That is non GUID value.
+    fs.setUserGroupRepresentationAsUPN(false);
+    fs.createNewFile(path);
+    Assert.assertTrue(fs.isFile(path));
+    FileStatus fileStatus = fs.getFileStatus(path);
+    UUID.fromString(fileStatus.getGroup());
+    UUID.fromString(fileStatus.getOwner());
+
+    // When set to false, User/Group information should be AAD represented
+    // unique OID. That is GUID value.
+    // Majority of the cases, user friendly name would not be GUID value.
+    fs.setUserGroupRepresentationAsUPN(true);
+    fileStatus = fs.getFileStatus(path);
+    try {
+      UUID.fromString(fileStatus.getGroup());
+      UUID.fromString(fileStatus.getOwner());
+      fail("Expected user friendly name to be non guid value.");
+    } catch (IllegalArgumentException e) {
+      // expected to fail since
+    }
+  }
 }
 


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


[35/50] [abbrv] hadoop git commit: HADOOP-13805. UGI.getCurrentUser() fails if user does not have a keytab associated. Contributed by Xiao Chen, Wei-Chiu Chuang, Yongjun Zhang.

Posted by xg...@apache.org.
HADOOP-13805. UGI.getCurrentUser() fails if user does not have a keytab associated. Contributed by Xiao Chen, Wei-Chiu Chuang, Yongjun Zhang.


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

Branch: refs/heads/YARN-5734
Commit: 4c26c241ad2b907dc02cecefa9846cbe2b0465ba
Parents: 02c5494
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Thu Feb 16 22:25:37 2017 -0800
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Fri Feb 17 09:18:50 2017 -0800

----------------------------------------------------------------------
 .../hadoop/fs/CommonConfigurationKeys.java      |  11 ++
 .../hadoop/security/UserGroupInformation.java   | 129 +++++++++++++++----
 .../hadoop/security/TestUGIWithMiniKdc.java     |   1 +
 .../security/TestUserGroupInformation.java      |  16 ++-
 4 files changed, 127 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c26c241/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
index b8a60d6..e53f71e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
@@ -353,6 +353,17 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   public static final String HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS =
     "hadoop.user.group.metrics.percentiles.intervals";
 
+  /* When creating UGI with UserGroupInformation(Subject), treat the passed
+   * subject external if set to true, and assume the owner of the subject
+   * should do the credential renewal.
+   *
+   * This is a temporary config to solve the compatibility issue with
+   * HADOOP-13558 and HADOOP-13805 fix, see the jiras for discussions.
+   */
+  public static final String HADOOP_TREAT_SUBJECT_EXTERNAL_KEY =
+      "hadoop.treat.subject.external";
+  public static final boolean HADOOP_TREAT_SUBJECT_EXTERNAL_DEFAULT = false;
+
   public static final String RPC_METRICS_QUANTILE_ENABLE =
       "rpc.metrics.quantile.enable";
   public static final boolean RPC_METRICS_QUANTILE_ENABLE_DEFAULT = false;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c26c241/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 6574e55..a5c6226 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
@@ -18,6 +18,8 @@
 package org.apache.hadoop.security;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS;
+import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_TREAT_SUBJECT_EXTERNAL_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_TREAT_SUBJECT_EXTERNAL_DEFAULT;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN_DEFAULT;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_TOKEN_FILES;
@@ -79,6 +81,7 @@ import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -273,6 +276,29 @@ public class UserGroupInformation {
   /** Min time (in seconds) before relogin for Kerberos */
   private static long kerberosMinSecondsBeforeRelogin;
   /** The configuration to use */
+
+  /*
+   * This config is a temporary one for backward compatibility.
+   * It means whether to treat the subject passed to
+   * UserGroupInformation(Subject) as external. If true,
+   * -  no renewal thread will be created to do the renew credential
+   * -  reloginFromKeytab() and reloginFromTicketCache will not renew
+   *    credential.
+   * and it assumes that the owner of the subject to renew; if false, it means
+   * to retain the old behavior prior to fixing HADOOP-13558 and HADOOP-13805.
+   * The default is false.
+   */
+  private static boolean treatSubjectExternal = false;
+
+  /*
+   * Some test need the renewal thread to be created even if it does
+   *   UserGroupInformation.loginUserFromSubject(subject);
+   * The test code may set this variable to true via
+   *   setEnableRenewThreadCreationForTest(boolean)
+   * method.
+   */
+  private static boolean enableRenewThreadCreationForTest = false;
+
   private static Configuration conf;
 
   
@@ -338,6 +364,15 @@ public class UserGroupInformation {
         metrics.getGroupsQuantiles = getGroupsQuantiles;
       }
     }
+
+    treatSubjectExternal = conf.getBoolean(HADOOP_TREAT_SUBJECT_EXTERNAL_KEY,
+        HADOOP_TREAT_SUBJECT_EXTERNAL_DEFAULT);
+    if (treatSubjectExternal) {
+      LOG.info("Config " + HADOOP_TREAT_SUBJECT_EXTERNAL_KEY + " is set to "
+          + "true, the owner of the subject passed to "
+          + " UserGroupInformation(Subject) is supposed to renew the "
+          + "credential.");
+    }
   }
 
   /**
@@ -351,7 +386,19 @@ public class UserGroupInformation {
   public static void setConfiguration(Configuration conf) {
     initialize(conf, true);
   }
-  
+
+  @InterfaceAudience.Private
+  @VisibleForTesting
+  static void setEnableRenewThreadCreationForTest(boolean b) {
+    enableRenewThreadCreationForTest = b;
+  }
+
+  @InterfaceAudience.Private
+  @VisibleForTesting
+  static boolean getEnableRenewThreadCreationForTest() {
+    return enableRenewThreadCreationForTest;
+  }
+
   @InterfaceAudience.Private
   @VisibleForTesting
   public static void reset() {
@@ -361,6 +408,7 @@ public class UserGroupInformation {
     kerberosMinSecondsBeforeRelogin = 0;
     setLoginUser(null);
     HadoopKerberosName.setRules(null);
+    setEnableRenewThreadCreationForTest(false);
   }
   
   /**
@@ -392,6 +440,7 @@ public class UserGroupInformation {
   private final User user;
   private final boolean isKeytab;
   private final boolean isKrbTkt;
+  private final boolean isLoginExternal;
   
   private static String OS_LOGIN_MODULE_NAME;
   private static Class<? extends Principal> OS_PRINCIPAL_CLASS;
@@ -644,28 +693,28 @@ public class UserGroupInformation {
   /**
    * Create a UserGroupInformation for the given subject.
    * This does not change the subject or acquire new credentials.
+   *
+   * The creator of subject is responsible for renewing credentials.
    * @param subject the user's subject
    */
   UserGroupInformation(Subject subject) {
-    this(subject, false);
+    this(subject, treatSubjectExternal);
   }
 
   /**
    * Create a UGI from the given subject.
    * @param subject the subject
-   * @param externalKeyTab if the subject's keytab is managed by the user.
+   * @param isLoginExternal if the subject's keytab is managed by other UGI.
    *                       Setting this to true will prevent UGI from attempting
    *                       to login the keytab, or to renew it.
    */
-  private UserGroupInformation(Subject subject, final boolean externalKeyTab) {
+  private UserGroupInformation(Subject subject, final boolean isLoginExternal) {
     this.subject = subject;
     this.user = subject.getPrincipals(User.class).iterator().next();
-    if (externalKeyTab) {
-      this.isKeytab = false;
-    } else {
-      this.isKeytab = KerberosUtil.hasKerberosKeyTab(subject);
-    }
+
+    this.isKeytab = KerberosUtil.hasKerberosKeyTab(subject);
     this.isKrbTkt = KerberosUtil.hasKerberosTicket(subject);
+    this.isLoginExternal = isLoginExternal;
   }
   
   /**
@@ -766,7 +815,7 @@ public class UserGroupInformation {
       User ugiUser = new User(loginPrincipals.iterator().next().getName(),
           AuthenticationMethod.KERBEROS, login);
       loginSubject.getPrincipals().add(ugiUser);
-      UserGroupInformation ugi = new UserGroupInformation(loginSubject);
+      UserGroupInformation ugi = new UserGroupInformation(loginSubject, false);
       ugi.setLogin(login);
       ugi.setAuthenticationMethod(AuthenticationMethod.KERBEROS);
       return ugi;
@@ -782,7 +831,9 @@ public class UserGroupInformation {
   /**
    * Create a UserGroupInformation from a Subject with Kerberos principal.
    *
-   * @param subject             The KerberosPrincipal to use in UGI
+   * @param subject             The KerberosPrincipal to use in UGI.
+   *                            The creator of subject is responsible for
+   *                            renewing credentials.
    *
    * @throws IOException
    * @throws KerberosAuthException if the kerberos login fails
@@ -843,6 +894,10 @@ public class UserGroupInformation {
    * Log in a user using the given subject
    * @param subject the subject to use when logging in a user, or null to
    * create a new subject.
+   *
+   * If subject is not null, the creator of subject is responsible for renewing
+   * credentials.
+   *
    * @throws IOException if login fails
    */
   @InterfaceAudience.Public
@@ -850,17 +905,25 @@ public class UserGroupInformation {
   public synchronized 
   static void loginUserFromSubject(Subject subject) throws IOException {
     ensureInitialized();
+    boolean externalSubject = false;
     try {
       if (subject == null) {
         subject = new Subject();
+      } else {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Treat subject external: " + treatSubjectExternal
+              + ". When true, assuming keytab is managed extenally since "
+              + " logged in from subject");
+        }
+        externalSubject = treatSubjectExternal;
       }
       LoginContext login =
           newLoginContext(authenticationMethod.getLoginAppName(), 
                           subject, new HadoopConfiguration());
       login.login();
-      LOG.debug("Assuming keytab is managed externally since logged in from"
-          + " subject.");
-      UserGroupInformation realUser = new UserGroupInformation(subject, true);
+
+      UserGroupInformation realUser =
+          new UserGroupInformation(subject, externalSubject);
       realUser.setLogin(login);
       realUser.setAuthenticationMethod(authenticationMethod);
       // If the HADOOP_PROXY_USER environment variable or property
@@ -959,11 +1022,23 @@ public class UserGroupInformation {
     return start + (long) ((end - start) * TICKET_RENEW_WINDOW);
   }
 
+  /**
+   * Should relogin if security is enabled using Kerberos, and
+   * the Subject is not owned by another UGI.
+   * @return true if this UGI should relogin
+   */
+  private boolean shouldRelogin() {
+    return isSecurityEnabled()
+        && user.getAuthenticationMethod() == AuthenticationMethod.KERBEROS
+        && !isLoginExternal;
+  }
+
   /**Spawn a thread to do periodic renewals of kerberos credentials*/
   private void spawnAutoRenewalThreadForUserCreds() {
-    if (!isSecurityEnabled()
-        || user.getAuthenticationMethod() != AuthenticationMethod.KERBEROS
-        || isKeytab) {
+    if (getEnableRenewThreadCreationForTest()) {
+      LOG.warn("Spawning thread to auto renew user credential since " +
+          " enableRenewThreadCreationForTest was set to true.");
+    } else if (!shouldRelogin() || isKeytab) {
       return;
     }
 
@@ -1092,7 +1167,7 @@ public class UserGroupInformation {
       start = Time.now();
       login.login();
       metrics.loginSuccess.add(Time.now() - start);
-      loginUser = new UserGroupInformation(subject);
+      loginUser = new UserGroupInformation(subject, false);
       loginUser.setLogin(login);
       loginUser.setAuthenticationMethod(AuthenticationMethod.KERBEROS);
     } catch (LoginException le) {
@@ -1156,8 +1231,9 @@ public class UserGroupInformation {
   public synchronized void checkTGTAndReloginFromKeytab() throws IOException {
     if (!isSecurityEnabled()
         || user.getAuthenticationMethod() != AuthenticationMethod.KERBEROS
-        || !isKeytab)
+        || !isKeytab) {
       return;
+    }
     KerberosTicket tgt = getTGT();
     if (tgt != null && !shouldRenewImmediatelyForTests &&
         Time.now() < getRefreshTime(tgt)) {
@@ -1210,9 +1286,7 @@ public class UserGroupInformation {
   @InterfaceAudience.Public
   @InterfaceStability.Evolving
   public synchronized void reloginFromKeytab() throws IOException {
-    if (!isSecurityEnabled()
-        || user.getAuthenticationMethod() != AuthenticationMethod.KERBEROS
-        || !isKeytab) {
+    if (!shouldRelogin() || !isKeytab) {
       return;
     }
 
@@ -1281,9 +1355,7 @@ public class UserGroupInformation {
   @InterfaceAudience.Public
   @InterfaceStability.Evolving
   public synchronized void reloginFromTicketCache() throws IOException {
-    if (!isSecurityEnabled()
-        || user.getAuthenticationMethod() != AuthenticationMethod.KERBEROS
-        || !isKrbTkt) {
+    if (!shouldRelogin() || !isKrbTkt) {
       return;
     }
     LoginContext login = getLogin();
@@ -1354,7 +1426,8 @@ public class UserGroupInformation {
       start = Time.now();
       login.login();
       metrics.loginSuccess.add(Time.now() - start);
-      UserGroupInformation newLoginUser = new UserGroupInformation(subject);
+      UserGroupInformation newLoginUser =
+          new UserGroupInformation(subject, false);
       newLoginUser.setLogin(login);
       newLoginUser.setAuthenticationMethod(AuthenticationMethod.KERBEROS);
       
@@ -1427,7 +1500,7 @@ public class UserGroupInformation {
     }
     Subject subject = new Subject();
     subject.getPrincipals().add(new User(user));
-    UserGroupInformation result = new UserGroupInformation(subject);
+    UserGroupInformation result = new UserGroupInformation(subject, false);
     result.setAuthenticationMethod(authMethod);
     return result;
   }
@@ -1504,7 +1577,7 @@ public class UserGroupInformation {
     Set<Principal> principals = subject.getPrincipals();
     principals.add(new User(user));
     principals.add(new RealUser(realUser));
-    UserGroupInformation result =new UserGroupInformation(subject);
+    UserGroupInformation result =new UserGroupInformation(subject, false);
     result.setAuthenticationMethod(AuthenticationMethod.PROXY);
     return result;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c26c241/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithMiniKdc.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithMiniKdc.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithMiniKdc.java
index 0016a65..2c6c7e4 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithMiniKdc.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithMiniKdc.java
@@ -75,6 +75,7 @@ public class TestUGIWithMiniKdc {
     SecurityUtil.setAuthenticationMethod(
         UserGroupInformation.AuthenticationMethod.KERBEROS, conf);
     UserGroupInformation.setConfiguration(conf);
+    UserGroupInformation.setEnableRenewThreadCreationForTest(true);
 
     LoginContext loginContext = null;
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c26c241/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
index ed0f58c..510987e 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
@@ -61,6 +61,7 @@ import java.util.LinkedHashSet;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_TREAT_SUBJECT_EXTERNAL_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTH_TO_LOCAL;
@@ -1020,8 +1021,7 @@ public class TestUserGroupInformation {
     assertTrue(credsugiTokens.contains(token2));
   }
 
-  @Test
-  public void testCheckTGTAfterLoginFromSubject() throws Exception {
+  private void testCheckTGTAfterLoginFromSubjectHelper() throws Exception {
     // security on, default is remove default realm
     SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf);
     UserGroupInformation.setConfiguration(conf);
@@ -1031,6 +1031,7 @@ public class TestUserGroupInformation {
     KeyTab keytab = KeyTab.getInstance();
     subject.getPrivateCredentials().add(keytab);
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+
     ugi.doAs(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws IOException {
@@ -1042,6 +1043,17 @@ public class TestUserGroupInformation {
     });
   }
 
+  @Test(expected = KerberosAuthException.class)
+  public void testCheckTGTAfterLoginFromSubject() throws Exception {
+    testCheckTGTAfterLoginFromSubjectHelper();
+  }
+
+  @Test
+  public void testCheckTGTAfterLoginFromSubjectFix() throws Exception {
+    conf.setBoolean(HADOOP_TREAT_SUBJECT_EXTERNAL_KEY, true);
+    testCheckTGTAfterLoginFromSubjectHelper();
+  }
+
   @Test
   public void testGetNextRetryTime() throws Exception {
     GenericTestUtils.setLogLevel(UserGroupInformation.LOG, Level.DEBUG);


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


[28/50] [abbrv] hadoop git commit: YARN-4675. Reorganize TimelineClient and TimelineClientImpl into separate classes for ATSv1.x and ATSv2. Contributed by Naganarasimha G R.

Posted by xg...@apache.org.
YARN-4675. Reorganize TimelineClient and TimelineClientImpl into separate classes for ATSv1.x and ATSv2. 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/4fa1afdb
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4fa1afdb
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4fa1afdb

Branch: refs/heads/YARN-5734
Commit: 4fa1afdb883dab8786d2fb5c72a195dd2e87d711
Parents: 5690b51
Author: Sangjin Lee <sj...@apache.org>
Authored: Thu Feb 16 11:41:04 2017 -0800
Committer: Sangjin Lee <sj...@apache.org>
Committed: Thu Feb 16 11:41:04 2017 -0800

----------------------------------------------------------------------
 .../jobhistory/JobHistoryEventHandler.java      |  57 +-
 .../hadoop/mapreduce/v2/app/MRAppMaster.java    |  14 +-
 .../v2/app/rm/RMContainerAllocator.java         |   4 +-
 .../jobhistory/TestJobHistoryEventHandler.java  |   8 +-
 .../distributedshell/ApplicationMaster.java     |  98 ++-
 .../hadoop/yarn/client/api/AMRMClient.java      |  40 +-
 .../yarn/client/api/async/AMRMClientAsync.java  |  21 +-
 .../api/async/impl/AMRMClientAsyncImpl.java     |   5 +-
 .../yarn/client/api/impl/YarnClientImpl.java    |  15 +-
 .../hadoop/yarn/client/api/TimelineClient.java  |  94 +--
 .../yarn/client/api/TimelineV2Client.java       |  92 +++
 .../client/api/impl/TimelineClientImpl.java     | 825 ++-----------------
 .../yarn/client/api/impl/TimelineConnector.java | 440 ++++++++++
 .../client/api/impl/TimelineV2ClientImpl.java   | 459 +++++++++++
 .../client/api/impl/TestTimelineClient.java     |  39 +-
 .../api/impl/TestTimelineClientV2Impl.java      |   4 +-
 .../timelineservice/NMTimelinePublisher.java    |  22 +-
 .../TestNMTimelinePublisher.java                |  10 +-
 .../TestTimelineServiceClientIntegration.java   |  10 +-
 19 files changed, 1272 insertions(+), 985 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
index 0cc605c..285d36e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
@@ -72,13 +72,12 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.JsonNodeFactory;
 import com.google.common.annotations.VisibleForTesting;
 import com.sun.jersey.api.client.ClientHandlerException;
 
@@ -90,8 +89,6 @@ import com.sun.jersey.api.client.ClientHandlerException;
  */
 public class JobHistoryEventHandler extends AbstractService
     implements EventHandler<JobHistoryEvent> {
-  private static final JsonNodeFactory FACTORY =
-      new ObjectMapper().getNodeFactory();
 
   private final AppContext context;
   private final int startCount;
@@ -133,9 +130,10 @@ public class JobHistoryEventHandler extends AbstractService
   // should job completion be force when the AM shuts down?
   protected volatile boolean forceJobCompletion = false;
 
+  @VisibleForTesting
   protected TimelineClient timelineClient;
-
-  private boolean timelineServiceV2Enabled = false;
+  @VisibleForTesting
+  protected TimelineV2Client timelineV2Client;
 
   private static String MAPREDUCE_JOB_ENTITY_TYPE = "MAPREDUCE_JOB";
   private static String MAPREDUCE_TASK_ENTITY_TYPE = "MAPREDUCE_TASK";
@@ -268,12 +266,17 @@ public class JobHistoryEventHandler extends AbstractService
         MRJobConfig.DEFAULT_MAPREDUCE_JOB_EMIT_TIMELINE_DATA)) {
       LOG.info("Emitting job history data to the timeline service is enabled");
       if (YarnConfiguration.timelineServiceEnabled(conf)) {
-
-        timelineClient =
-            ((MRAppMaster.RunningAppContext)context).getTimelineClient();
-        timelineClient.init(conf);
-        timelineServiceV2Enabled =
-            YarnConfiguration.timelineServiceV2Enabled(conf);
+        boolean timelineServiceV2Enabled =
+            ((int) YarnConfiguration.getTimelineServiceVersion(conf) == 2);
+        if(timelineServiceV2Enabled) {
+          timelineV2Client =
+              ((MRAppMaster.RunningAppContext)context).getTimelineV2Client();
+          timelineV2Client.init(conf);
+        } else {
+          timelineClient =
+              ((MRAppMaster.RunningAppContext) context).getTimelineClient();
+          timelineClient.init(conf);
+        }
         LOG.info("Timeline service is enabled; version: " +
             YarnConfiguration.getTimelineServiceVersion(conf));
       } else {
@@ -324,6 +327,8 @@ public class JobHistoryEventHandler extends AbstractService
   protected void serviceStart() throws Exception {
     if (timelineClient != null) {
       timelineClient.start();
+    } else if (timelineV2Client != null) {
+      timelineV2Client.start();
     }
     eventHandlingThread = new Thread(new Runnable() {
       @Override
@@ -448,6 +453,8 @@ public class JobHistoryEventHandler extends AbstractService
     }
     if (timelineClient != null) {
       timelineClient.stop();
+    } else if (timelineV2Client != null) {
+      timelineV2Client.stop();
     }
     LOG.info("Stopped JobHistoryEventHandler. super.stop()");
     super.serviceStop();
@@ -605,14 +612,12 @@ public class JobHistoryEventHandler extends AbstractService
         }
         processEventForJobSummary(event.getHistoryEvent(), mi.getJobSummary(),
             event.getJobID());
-        if (timelineClient != null) {
-          if (timelineServiceV2Enabled) {
-            processEventForNewTimelineService(historyEvent, event.getJobID(),
-                event.getTimestamp());
-          } else {
-            processEventForTimelineServer(historyEvent, event.getJobID(),
-                event.getTimestamp());
-          }
+        if (timelineV2Client != null) {
+          processEventForNewTimelineService(historyEvent, event.getJobID(),
+              event.getTimestamp());
+        } else if (timelineClient != null) {
+          processEventForTimelineServer(historyEvent, event.getJobID(),
+              event.getTimestamp());
         }
         if (LOG.isDebugEnabled()) {
           LOG.debug("In HistoryEventHandler "
@@ -1162,8 +1167,8 @@ public class JobHistoryEventHandler extends AbstractService
         configSize += size;
         if (configSize > JobHistoryEventUtils.ATS_CONFIG_PUBLISH_SIZE_BYTES) {
           if (jobEntityForConfigs.getConfigs().size() > 0) {
-            timelineClient.putEntities(jobEntityForConfigs);
-            timelineClient.putEntities(appEntityForConfigs);
+            timelineV2Client.putEntities(jobEntityForConfigs);
+            timelineV2Client.putEntities(appEntityForConfigs);
             jobEntityForConfigs = createJobEntity(jobId);
             appEntityForConfigs = new ApplicationEntity();
             appEntityForConfigs.setId(appId);
@@ -1174,8 +1179,8 @@ public class JobHistoryEventHandler extends AbstractService
         appEntityForConfigs.addConfig(entry.getKey(), entry.getValue());
       }
       if (configSize > 0) {
-        timelineClient.putEntities(jobEntityForConfigs);
-        timelineClient.putEntities(appEntityForConfigs);
+        timelineV2Client.putEntities(jobEntityForConfigs);
+        timelineV2Client.putEntities(appEntityForConfigs);
       }
     } catch (IOException | YarnException e) {
       LOG.error("Exception while publishing configs on JOB_SUBMITTED Event " +
@@ -1295,9 +1300,9 @@ public class JobHistoryEventHandler extends AbstractService
     }
     try {
       if (appEntityWithJobMetrics == null) {
-        timelineClient.putEntitiesAsync(tEntity);
+        timelineV2Client.putEntitiesAsync(tEntity);
       } else {
-        timelineClient.putEntities(tEntity, appEntityWithJobMetrics);
+        timelineV2Client.putEntities(tEntity, appEntityWithJobMetrics);
       }
     } catch (IOException | YarnException e) {
       LOG.error("Failed to process Event " + event.getEventType()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
index 835c0aa..12df83d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
@@ -38,6 +38,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import javax.crypto.KeyGenerator;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -141,6 +143,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -154,8 +157,6 @@ import org.apache.hadoop.yarn.util.SystemClock;
 
 import com.google.common.annotations.VisibleForTesting;
 
-import javax.crypto.KeyGenerator;
-
 /**
  * The Map-Reduce Application Master.
  * The state machine is encapsulated in the implementation of Job interface.
@@ -1066,6 +1067,7 @@ public class MRAppMaster extends CompositeService {
     private final ClusterInfo clusterInfo = new ClusterInfo();
     private final ClientToAMTokenSecretManager clientToAMTokenSecretManager;
     private TimelineClient timelineClient = null;
+    private TimelineV2Client timelineV2Client = null;
 
     private final TaskAttemptFinishingMonitor taskAttemptFinishingMonitor;
 
@@ -1081,7 +1083,7 @@ public class MRAppMaster extends CompositeService {
 
         if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
           // create new version TimelineClient
-          timelineClient = TimelineClient.createTimelineClient(
+          timelineV2Client = TimelineV2Client.createTimelineClient(
               appAttemptID.getApplicationId());
         } else {
           timelineClient = TimelineClient.createTimelineClient();
@@ -1177,10 +1179,14 @@ public class MRAppMaster extends CompositeService {
       return taskAttemptFinishingMonitor;
     }
 
-    // Get Timeline Collector's address (get sync from RM)
     public TimelineClient getTimelineClient() {
       return timelineClient;
     }
+
+    // Get Timeline Collector's address (get sync from RM)
+    public TimelineV2Client getTimelineV2Client() {
+      return timelineV2Client;
+    }
   }
 
   @SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
index 31bc380..1f88a2c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
@@ -882,8 +882,8 @@ public class RMContainerAllocator extends RMContainerRequestor
     MRAppMaster.RunningAppContext appContext =
         (MRAppMaster.RunningAppContext)this.getContext();
     if (collectorAddr != null && !collectorAddr.isEmpty()
-        && appContext.getTimelineClient() != null) {
-      appContext.getTimelineClient().setTimelineServiceAddress(
+        && appContext.getTimelineV2Client() != null) {
+      appContext.getTimelineV2Client().setTimelineServiceAddress(
           response.getCollectorAddr());
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
index 0b33d6b..6c5e604 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
@@ -29,8 +29,8 @@ import static org.mockito.Mockito.when;
 
 import java.io.File;
 import java.io.FileOutputStream;
-import java.io.InputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.util.HashMap;
 
 import org.apache.commons.logging.Log;
@@ -69,6 +69,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
@@ -829,6 +830,9 @@ public class TestJobHistoryEventHandler {
     if (mockContext instanceof RunningAppContext) {
       when(((RunningAppContext)mockContext).getTimelineClient()).
           thenReturn(TimelineClient.createTimelineClient());
+      when(((RunningAppContext) mockContext).getTimelineV2Client())
+          .thenReturn(TimelineV2Client
+              .createTimelineClient(ApplicationId.newInstance(0, 1)));
     }
     return mockContext;
   }
@@ -937,6 +941,8 @@ class JHEvenHandlerForTest extends JobHistoryEventHandler {
   protected void serviceStart() {
     if (timelineClient != null) {
       timelineClient.start();
+    } else if (timelineV2Client != null) {
+      timelineV2Client.start();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
index 5a06ef6..4daebb5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
@@ -97,6 +97,7 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
 import org.apache.hadoop.yarn.client.api.async.NMClientAsync;
 import org.apache.hadoop.yarn.client.api.async.impl.NMClientAsyncImpl;
@@ -219,7 +220,9 @@ public class ApplicationMaster {
   // Tracking url to which app master publishes info for clients to monitor
   private String appMasterTrackingUrl = "";
 
-  private boolean timelineServiceV2 = false;
+  private boolean timelineServiceV2Enabled = false;
+
+  private boolean timelineServiceV1Enabled = false;
 
   // App Master configuration
   // No. of containers to run shell command on
@@ -293,6 +296,10 @@ public class ApplicationMaster {
   // Timeline Client
   @VisibleForTesting
   TimelineClient timelineClient;
+
+  // Timeline v2 Client
+  private TimelineV2Client timelineV2Client;
+
   static final String CONTAINER_ENTITY_GROUP_ID = "CONTAINERS";
   static final String APPID_TIMELINE_FILTER_NAME = "appId";
   static final String USER_TIMELINE_FILTER_NAME = "user";
@@ -556,9 +563,12 @@ public class ApplicationMaster {
         "container_retry_interval", "0"));
 
     if (YarnConfiguration.timelineServiceEnabled(conf)) {
-      timelineServiceV2 = YarnConfiguration.timelineServiceV2Enabled(conf);
+      timelineServiceV2Enabled =
+          ((int) YarnConfiguration.getTimelineServiceVersion(conf) == 2);
+      timelineServiceV1Enabled = !timelineServiceV2Enabled;
     } else {
       timelineClient = null;
+      timelineV2Client = null;
       LOG.warn("Timeline service is not enabled");
     }
 
@@ -621,18 +631,17 @@ public class ApplicationMaster {
     nmClientAsync.start();
 
     startTimelineClient(conf);
-    if (timelineServiceV2) {
+    if (timelineServiceV2Enabled) {
       // need to bind timelineClient
-      amRMClient.registerTimelineClient(timelineClient);
+      amRMClient.registerTimelineV2Client(timelineV2Client);
     }
-    if(timelineClient != null) {
-      if (timelineServiceV2) {
-        publishApplicationAttemptEventOnTimelineServiceV2(
-            DSEvent.DS_APP_ATTEMPT_START);
-      } else {
-        publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
-            DSEvent.DS_APP_ATTEMPT_START, domainId, appSubmitterUgi);
-      }
+
+    if (timelineServiceV2Enabled) {
+      publishApplicationAttemptEventOnTimelineServiceV2(
+          DSEvent.DS_APP_ATTEMPT_START);
+    } else if (timelineServiceV1Enabled) {
+      publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
+          DSEvent.DS_APP_ATTEMPT_START, domainId, appSubmitterUgi);
     }
 
     // Setup local RPC Server to accept status requests directly from clients
@@ -704,18 +713,21 @@ public class ApplicationMaster {
         public Void run() throws Exception {
           if (YarnConfiguration.timelineServiceEnabled(conf)) {
             // Creating the Timeline Client
-            if (timelineServiceV2) {
-              timelineClient = TimelineClient.createTimelineClient(
+            if (timelineServiceV2Enabled) {
+              timelineV2Client = TimelineV2Client.createTimelineClient(
                   appAttemptID.getApplicationId());
+              timelineV2Client.init(conf);
+              timelineV2Client.start();
               LOG.info("Timeline service V2 client is enabled");
             } else {
               timelineClient = TimelineClient.createTimelineClient();
+              timelineClient.init(conf);
+              timelineClient.start();
               LOG.info("Timeline service V1 client is enabled");
             }
-            timelineClient.init(conf);
-            timelineClient.start();
           } else {
             timelineClient = null;
+            timelineV2Client = null;
             LOG.warn("Timeline service is not enabled");
           }
           return null;
@@ -741,14 +753,12 @@ public class ApplicationMaster {
       } catch (InterruptedException ex) {}
     }
 
-    if (timelineClient != null) {
-      if (timelineServiceV2) {
-        publishApplicationAttemptEventOnTimelineServiceV2(
-            DSEvent.DS_APP_ATTEMPT_END);
-      } else {
-        publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
-            DSEvent.DS_APP_ATTEMPT_END, domainId, appSubmitterUgi);
-      }
+    if (timelineServiceV2Enabled) {
+      publishApplicationAttemptEventOnTimelineServiceV2(
+          DSEvent.DS_APP_ATTEMPT_END);
+    } else if (timelineServiceV1Enabled) {
+      publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
+          DSEvent.DS_APP_ATTEMPT_END, domainId, appSubmitterUgi);
     }
 
     // Join all launched threads
@@ -797,8 +807,10 @@ public class ApplicationMaster {
     amRMClient.stop();
 
     // Stop Timeline Client
-    if(timelineClient != null) {
+    if(timelineServiceV1Enabled) {
       timelineClient.stop();
+    } else if (timelineServiceV2Enabled) {
+      timelineV2Client.stop();
     }
 
     return success;
@@ -853,16 +865,14 @@ public class ApplicationMaster {
           LOG.info("Container completed successfully." + ", containerId="
               + containerStatus.getContainerId());
         }
-        if(timelineClient != null) {
-          if (timelineServiceV2) {
-            publishContainerEndEventOnTimelineServiceV2(containerStatus);
-          } else {
-            publishContainerEndEvent(
-                timelineClient, containerStatus, domainId, appSubmitterUgi);
-          }
+        if (timelineServiceV2Enabled) {
+          publishContainerEndEventOnTimelineServiceV2(containerStatus);
+        } else if (timelineServiceV1Enabled) {
+          publishContainerEndEvent(timelineClient, containerStatus, domainId,
+              appSubmitterUgi);
         }
       }
-      
+
       // ask for more containers if any failed
       int askCount = numTotalContainers - numRequestedContainers.get();
       numRequestedContainers.addAndGet(askCount);
@@ -983,15 +993,13 @@ public class ApplicationMaster {
         applicationMaster.nmClientAsync.getContainerStatusAsync(
             containerId, container.getNodeId());
       }
-      if(applicationMaster.timelineClient != null) {
-        if (applicationMaster.timelineServiceV2) {
-          applicationMaster.publishContainerStartEventOnTimelineServiceV2(
-              container);
-        } else {
-          applicationMaster.publishContainerStartEvent(
-              applicationMaster.timelineClient, container,
-              applicationMaster.domainId, applicationMaster.appSubmitterUgi);
-        }
+      if (applicationMaster.timelineServiceV2Enabled) {
+        applicationMaster
+            .publishContainerStartEventOnTimelineServiceV2(container);
+      } else if (applicationMaster.timelineServiceV1Enabled) {
+        applicationMaster.publishContainerStartEvent(
+            applicationMaster.timelineClient, container,
+            applicationMaster.domainId, applicationMaster.appSubmitterUgi);
       }
     }
 
@@ -1371,7 +1379,7 @@ public class ApplicationMaster {
       appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
         @Override
         public TimelinePutResponse run() throws Exception {
-          timelineClient.putEntities(entity);
+          timelineV2Client.putEntities(entity);
           return null;
         }
       });
@@ -1404,7 +1412,7 @@ public class ApplicationMaster {
       appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
         @Override
         public TimelinePutResponse run() throws Exception {
-          timelineClient.putEntities(entity);
+          timelineV2Client.putEntities(entity);
           return null;
         }
       });
@@ -1438,7 +1446,7 @@ public class ApplicationMaster {
       appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
         @Override
         public TimelinePutResponse run() throws Exception {
-          timelineClient.putEntitiesAsync(entity);
+          timelineV2Client.putEntitiesAsync(entity);
           return null;
         }
       });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
index 15d0065..69f3777 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
@@ -41,12 +42,13 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.resource.Resources;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
-import org.apache.hadoop.yarn.util.resource.Resources;
 
 @InterfaceAudience.Public
 @InterfaceStability.Stable
@@ -54,7 +56,8 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
     AbstractService {
   private static final Log LOG = LogFactory.getLog(AMRMClient.class);
 
-  private TimelineClient timelineClient;
+  private TimelineV2Client timelineV2Client;
+  private boolean timelineServiceV2Enabled;
 
   /**
    * Create a new instance of AMRMClient.
@@ -79,6 +82,12 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
     nmTokenCache = NMTokenCache.getSingleton();
   }
 
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    timelineServiceV2Enabled = YarnConfiguration.timelineServiceV2Enabled(conf);
+  }
+
   /**
    * Object to represent a single container request for resources. Scheduler
    * documentation should be consulted for the specifics of how the parameters
@@ -682,19 +691,30 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
   }
 
   /**
-   * Register TimelineClient to AMRMClient.
-   * @param client the timeline client to register
+   * Register TimelineV2Client to AMRMClient. Writer's address for the timeline
+   * V2 client will be updated dynamically if registered.
+   *
+   * @param client the timeline v2 client to register
+   * @throws YarnException when this method is invoked even when ATS V2 is not
+   *           configured.
    */
-  public void registerTimelineClient(TimelineClient client) {
-    this.timelineClient = client;
+  public void registerTimelineV2Client(TimelineV2Client client)
+      throws YarnException {
+    if (timelineServiceV2Enabled) {
+      timelineV2Client = client;
+    } else {
+      LOG.error("Trying to register timeline v2 client when not configured.");
+      throw new YarnException(
+          "register timeline v2 client when not configured.");
+    }
   }
 
   /**
-   * Get registered timeline client.
-   * @return the registered timeline client
+   * Get registered timeline v2 client.
+   * @return the registered timeline v2 client
    */
-  public TimelineClient getRegisteredTimelineClient() {
-    return this.timelineClient;
+  public TimelineV2Client getRegisteredTimelineV2Client() {
+    return this.timelineV2Client;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
index 4cb27cd..1ecfe1f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.yarn.client.api.async;
 
-import com.google.common.base.Preconditions;
-import com.google.common.base.Supplier;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
@@ -29,8 +27,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.Stable;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -46,13 +44,15 @@ import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.client.api.async.impl.AMRMClientAsyncImpl;
 import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
-import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.resource.Resources;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.yarn.util.resource.Resources;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
 
 /**
  * <code>AMRMClientAsync</code> handles communication with the ResourceManager
@@ -346,17 +346,20 @@ extends AbstractService {
   /**
    * Register TimelineClient to AMRMClient.
    * @param timelineClient
+   * @throws YarnException when this method is invoked even when ATS V2 is not
+   *           configured.
    */
-  public void registerTimelineClient(TimelineClient timelineClient) {
-    client.registerTimelineClient(timelineClient);
+  public void registerTimelineV2Client(TimelineV2Client timelineClient)
+      throws YarnException {
+    client.registerTimelineV2Client(timelineClient);
   }
 
   /**
    * Get registered timeline client.
    * @return the registered timeline client
    */
-  public TimelineClient getRegisteredTimelineClient() {
-    return client.getRegisteredTimelineClient();
+  public TimelineV2Client getRegisteredTimelineV2Client() {
+    return client.getRegisteredTimelineV2Client();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
index 9e2c0e5..6711da2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
-import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
 import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
 import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
@@ -326,7 +326,8 @@ extends AMRMClientAsync<T> {
 
           AllocateResponse response = (AllocateResponse) object;
           String collectorAddress = response.getCollectorAddr();
-          TimelineClient timelineClient = client.getRegisteredTimelineClient();
+          TimelineV2Client timelineClient =
+              client.getRegisteredTimelineV2Client();
           if (timelineClient != null && collectorAddress != null
               && !collectorAddress.isEmpty()) {
             if (collectorAddr == null

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
index e406862..4a27fee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
@@ -141,7 +141,7 @@ public class YarnClientImpl extends YarnClient {
   Text timelineService;
   @VisibleForTesting
   String timelineDTRenewer;
-  protected boolean timelineServiceEnabled;
+  private boolean timelineV1ServiceEnabled;
   protected boolean timelineServiceBestEffort;
 
   private static final String ROOT = "root";
@@ -167,9 +167,14 @@ public class YarnClientImpl extends YarnClient {
         YarnConfiguration.DEFAULT_YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS);
     }
 
+    float timelineServiceVersion =
+        conf.getFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION,
+            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_VERSION);
     if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
-        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) {
-      timelineServiceEnabled = true;
+        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)
+        && ((Float.compare(timelineServiceVersion, 1.0f) == 0)
+            || (Float.compare(timelineServiceVersion, 1.5f) == 0))) {
+      timelineV1ServiceEnabled = true;
       timelineDTRenewer = getTimelineDelegationTokenRenewer(conf);
       timelineService = TimelineUtils.buildTimelineTokenService(conf);
     }
@@ -178,7 +183,7 @@ public class YarnClientImpl extends YarnClient {
     // TimelineServer which means we are able to get history information
     // for applications/applicationAttempts/containers by using ahsClient
     // when the TimelineServer is running.
-    if (timelineServiceEnabled || conf.getBoolean(
+    if (timelineV1ServiceEnabled || conf.getBoolean(
         YarnConfiguration.APPLICATION_HISTORY_ENABLED,
         YarnConfiguration.DEFAULT_APPLICATION_HISTORY_ENABLED)) {
       historyServiceEnabled = true;
@@ -257,7 +262,7 @@ public class YarnClientImpl extends YarnClient {
 
     // Automatically add the timeline DT into the CLC
     // Only when the security and the timeline service are both enabled
-    if (isSecurityEnabled() && timelineServiceEnabled) {
+    if (isSecurityEnabled() && timelineV1ServiceEnabled) {
       addTimelineDelegationToken(appContext.getAMContainerSpec());
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/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 cc76718..4835239 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
@@ -21,14 +21,12 @@ package org.apache.hadoop.yarn.client.api;
 import java.io.Flushable;
 import java.io.IOException;
 
-import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 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;
+import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId;
@@ -39,24 +37,22 @@ import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
 
 /**
  * A client library that can be used to post some information in terms of a
- * number of conceptual entities.
+ * number of conceptual entities. This client library needs to be used along
+ * with Timeline V.1.x server versions.
+ * Refer {@link TimelineV2Client} for ATS V2 interface.
  */
 @Public
 @Evolving
-public abstract class TimelineClient extends AbstractService implements
+public abstract class TimelineClient extends CompositeService implements
     Flushable {
 
   /**
-   * Create a timeline client. The current UGI when the user initialize the
-   * client will be used to do the put and the delegation token operations. The
-   * current user may use {@link UserGroupInformation#doAs} another user to
-   * construct and initialize a timeline client if the following operations are
-   * supposed to be conducted by that user.
-   */
-  private ApplicationId contextAppId;
-
-  /**
    * Creates an instance of the timeline v.1.x client.
+   * The current UGI when the user initialize the client will be used to do the
+   * put and the delegation token operations. The current user may use
+   * {@link UserGroupInformation#doAs} another user to construct and initialize
+   * a timeline client if the following operations are supposed to be conducted
+   * by that user.
    *
    * @return the created timeline client instance
    */
@@ -66,23 +62,8 @@ public abstract class TimelineClient extends AbstractService implements
     return client;
   }
 
-  /**
-   * Creates an instance of the timeline v.2 client.
-   *
-   * @param appId the application id with which the timeline client is
-   * associated
-   * @return the created timeline client instance
-   */
-  @Public
-  public static TimelineClient createTimelineClient(ApplicationId appId) {
-    TimelineClient client = new TimelineClientImpl(appId);
-    return client;
-  }
-
-  @Private
-  protected TimelineClient(String name, ApplicationId appId) {
+  protected TimelineClient(String name) {
     super(name);
-    setContextAppId(appId);
   }
 
   /**
@@ -207,57 +188,4 @@ public abstract class TimelineClient extends AbstractService implements
   public abstract void cancelDelegationToken(
       Token<TimelineDelegationTokenIdentifier> timelineDT)
           throws IOException, YarnException;
-
-  /**
-   * <p>
-   * Send the information of a number of conceptual entities to the timeline
-   * service v.2 collector. It is a blocking API. The method will not return
-   * until all the put entities have been persisted. If this method is invoked
-   * for a non-v.2 timeline client instance, a YarnException is thrown.
-   * </p>
-   *
-   * @param entities the collection of {@link
-   * org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity}
-   * @throws IOException
-   * @throws YarnException
-   */
-  @Public
-  public abstract void putEntities(
-      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity...
-          entities) throws IOException, YarnException;
-
-  /**
-   * <p>
-   * Send the information of a number of conceptual entities to the timeline
-   * service v.2 collector. It is an asynchronous API. The method will return
-   * once all the entities are received. If this method is invoked for a
-   * non-v.2 timeline client instance, a YarnException is thrown.
-   * </p>
-   *
-   * @param entities the collection of {@link
-   * org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity}
-   * @throws IOException
-   * @throws YarnException
-   */
-  @Public
-  public abstract void putEntitiesAsync(
-      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity...
-          entities) throws IOException, YarnException;
-
-  /**
-   * <p>
-   * Update the timeline service address where the request will be sent to.
-   * </p>
-   * @param address
-   *          the timeline service address
-   */
-  public abstract void setTimelineServiceAddress(String address);
-
-  protected ApplicationId getContextAppId() {
-    return contextAppId;
-  }
-
-  protected void setContextAppId(ApplicationId appId) {
-    this.contextAppId = appId;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineV2Client.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineV2Client.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineV2Client.java
new file mode 100644
index 0000000..32cf1e9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineV2Client.java
@@ -0,0 +1,92 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.client.api;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.client.api.impl.TimelineV2ClientImpl;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * A client library that can be used to post some information in terms of a
+ * number of conceptual entities. This client library needs to be used along
+ * with time line v.2 server version.
+ * Refer {@link TimelineClient} for ATS V1 interface.
+ */
+public abstract class TimelineV2Client extends CompositeService {
+  /**
+   * Creates an instance of the timeline v.2 client.
+   *
+   * @param appId the application id with which the timeline client is
+   *          associated
+   * @return the created timeline client instance
+   */
+  @Public
+  public static TimelineV2Client createTimelineClient(ApplicationId appId) {
+    TimelineV2Client client = new TimelineV2ClientImpl(appId);
+    return client;
+  }
+
+  protected TimelineV2Client(String name) {
+    super(name);
+  }
+
+  /**
+   * <p>
+   * Send the information of a number of conceptual entities to the timeline
+   * service v.2 collector. It is a blocking API. The method will not return
+   * until all the put entities have been persisted.
+   * </p>
+   *
+   * @param entities the collection of {@link TimelineEntity}
+   * @throws IOException  if there are I/O errors
+   * @throws YarnException if entities are incomplete/invalid
+   */
+  @Public
+  public abstract void putEntities(TimelineEntity... entities)
+      throws IOException, YarnException;
+
+  /**
+   * <p>
+   * Send the information of a number of conceptual entities to the timeline
+   * service v.2 collector. It is an asynchronous API. The method will return
+   * once all the entities are received.
+   * </p>
+   *
+   * @param entities the collection of {@link TimelineEntity}
+   * @throws IOException  if there are I/O errors
+   * @throws YarnException if entities are incomplete/invalid
+   */
+  @Public
+  public abstract void putEntitiesAsync(TimelineEntity... entities)
+      throws IOException, YarnException;
+
+  /**
+   * <p>
+   * Update the timeline service address where the request will be sent to.
+   * </p>
+   *
+   * @param address the timeline service address
+   */
+  public abstract void setTimelineServiceAddress(String address);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4fa1afdb/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 4506c48..f49618b 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
@@ -20,32 +20,10 @@ package org.apache.hadoop.yarn.client.api.impl;
 
 import java.io.File;
 import java.io.IOException;
-import java.lang.reflect.UndeclaredThrowableException;
-import java.net.ConnectException;
-import java.net.HttpURLConnection;
 import java.net.InetSocketAddress;
-import java.net.SocketTimeoutException;
 import java.net.URI;
-import java.net.URL;
-import java.net.URLConnection;
-import java.security.GeneralSecurityException;
 import java.security.PrivilegedExceptionAction;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.FutureTask;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
 
-import javax.net.ssl.HostnameVerifier;
-import javax.net.ssl.HttpsURLConnection;
-import javax.net.ssl.SSLSocketFactory;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.MultivaluedMap;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.HelpFormatter;
@@ -57,16 +35,9 @@ import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authentication.client.AuthenticationException;
-import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
-import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
-import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator;
-import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticator;
-import org.apache.hadoop.security.token.delegation.web.PseudoDelegationTokenAuthenticator;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
@@ -79,19 +50,9 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
 import com.sun.jersey.api.client.Client;
-import com.sun.jersey.api.client.ClientHandlerException;
-import com.sun.jersey.api.client.ClientRequest;
-import com.sun.jersey.api.client.ClientResponse;
-import com.sun.jersey.api.client.config.ClientConfig;
-import com.sun.jersey.api.client.config.DefaultClientConfig;
-import com.sun.jersey.api.client.filter.ClientFilter;
-import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
-import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
-import com.sun.jersey.core.util.MultivaluedMapImpl;
 
 @Private
 @Evolving
@@ -100,9 +61,6 @@ public class TimelineClientImpl extends TimelineClient {
   private static final Log LOG = LogFactory.getLog(TimelineClientImpl.class);
   private static final ObjectMapper MAPPER = new ObjectMapper();
   private static final String RESOURCE_URI_STR_V1 = "/ws/v1/timeline/";
-  private static final String RESOURCE_URI_STR_V2 = "/ws/v2/timeline/";
-  private static final Joiner JOINER = Joiner.on("");
-  public final static int DEFAULT_SOCKET_TIMEOUT = 1 * 60 * 1000; // 1 minute
 
   private static Options opts;
   private static final String ENTITY_DATA_TYPE = "entity";
@@ -117,179 +75,38 @@ public class TimelineClientImpl extends TimelineClient {
     opts.addOption("help", false, "Print usage");
   }
 
-  private Client client;
-  private ConnectionConfigurator connConfigurator;
-  private DelegationTokenAuthenticator authenticator;
-  private DelegationTokenAuthenticatedURL.Token token;
-  private UserGroupInformation authUgi;
-  private String doAsUser;
-  private Configuration configuration;
-  private float timelineServiceVersion;
-  private TimelineWriter timelineWriter;
-  private SSLFactory sslFactory;
-
-  private volatile String timelineServiceAddress;
-
-  // Retry parameters for identifying new timeline service
-  // TODO consider to merge with connection retry
-  private int maxServiceRetries;
-  private long serviceRetryInterval;
-  private boolean timelineServiceV2 = false;
-
-  @Private
   @VisibleForTesting
-  TimelineClientConnectionRetry connectionRetry;
-
-  private TimelineEntityDispatcher entityDispatcher;
-
-  // Abstract class for an operation that should be retried by timeline client
-  @Private
+  protected DelegationTokenAuthenticatedURL.Token token;
   @VisibleForTesting
-  public static abstract class TimelineClientRetryOp {
-    // The operation that should be retried
-    public abstract Object run() throws IOException;
-    // The method to indicate if we should retry given the incoming exception
-    public abstract boolean shouldRetryOn(Exception e);
-  }
-
-  // Class to handle retry
-  // Outside this class, only visible to tests
-  @Private
+  protected UserGroupInformation authUgi;
   @VisibleForTesting
-  static class TimelineClientConnectionRetry {
-
-    // maxRetries < 0 means keep trying
-    @Private
-    @VisibleForTesting
-    public int maxRetries;
-
-    @Private
-    @VisibleForTesting
-    public long retryInterval;
-
-    // Indicates if retries happened last time. Only tests should read it.
-    // In unit tests, retryOn() calls should _not_ be concurrent.
-    private boolean retried = false;
+  protected String doAsUser;
 
-    @Private
-    @VisibleForTesting
-    boolean getRetired() {
-      return retried;
-    }
-
-    // Constructor with default retry settings
-    public TimelineClientConnectionRetry(Configuration conf) {
-      Preconditions.checkArgument(conf.getInt(
-          YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
-          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES) >= -1,
-          "%s property value should be greater than or equal to -1",
-          YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES);
-      Preconditions
-          .checkArgument(
-              conf.getLong(
-                  YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
-                  YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS) > 0,
-              "%s property value should be greater than zero",
-              YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS);
-      maxRetries = conf.getInt(
-        YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
-        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES);
-      retryInterval = conf.getLong(
-        YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
-        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS);
-    }
-
-    public Object retryOn(TimelineClientRetryOp op)
-        throws RuntimeException, IOException {
-      int leftRetries = maxRetries;
-      retried = false;
-
-      // keep trying
-      while (true) {
-        try {
-          // try perform the op, if fail, keep retrying
-          return op.run();
-        } catch (IOException | RuntimeException e) {
-          // break if there's no retries left
-          if (leftRetries == 0) {
-            break;
-          }
-          if (op.shouldRetryOn(e)) {
-            logException(e, leftRetries);
-          } else {
-            throw e;
-          }
-        }
-        if (leftRetries > 0) {
-          leftRetries--;
-        }
-        retried = true;
-        try {
-          // sleep for the given time interval
-          Thread.sleep(retryInterval);
-        } catch (InterruptedException ie) {
-          LOG.warn("Client retry sleep interrupted! ");
-        }
-      }
-      throw new RuntimeException("Failed to connect to timeline server. "
-          + "Connection retries limit exceeded. "
-          + "The posted timeline event may be missing");
-    };
-
-    private void logException(Exception e, int leftRetries) {
-      if (leftRetries > 0) {
-        LOG.info("Exception caught by TimelineClientConnectionRetry,"
-              + " will try " + leftRetries + " more time(s).\nMessage: "
-              + e.getMessage());
-      } else {
-        // note that maxRetries may be -1 at the very beginning
-        LOG.info("ConnectionException caught by TimelineClientConnectionRetry,"
-            + " will keep retrying.\nMessage: "
-            + e.getMessage());
-      }
-    }
-  }
+  private float timelineServiceVersion;
+  private TimelineWriter timelineWriter;
 
-  private class TimelineJerseyRetryFilter extends ClientFilter {
-    @Override
-    public ClientResponse handle(final ClientRequest cr)
-        throws ClientHandlerException {
-      // Set up the retry operation
-      TimelineClientRetryOp jerseyRetryOp = new TimelineClientRetryOp() {
-        @Override
-        public Object run() {
-          // Try pass the request, if fail, keep retrying
-          return getNext().handle(cr);
-        }
+  private String timelineServiceAddress;
 
-        @Override
-        public boolean shouldRetryOn(Exception e) {
-          // Only retry on connection exceptions
-          return (e instanceof ClientHandlerException)
-              && (e.getCause() instanceof ConnectException ||
-                  e.getCause() instanceof SocketTimeoutException);
-        }
-      };
-      try {
-        return (ClientResponse) connectionRetry.retryOn(jerseyRetryOp);
-      } catch (IOException e) {
-        throw new ClientHandlerException("Jersey retry failed!\nMessage: "
-              + e.getMessage());
-      }
-    }
-  }
+  @Private
+  @VisibleForTesting
+  TimelineConnector connector;
 
   public TimelineClientImpl() {
-    super(TimelineClientImpl.class.getName(), null);
-  }
-
-  public TimelineClientImpl(ApplicationId applicationId) {
-    super(TimelineClientImpl.class.getName(), applicationId);
-    this.timelineServiceV2 = true;
+    super(TimelineClientImpl.class.getName());
   }
 
   protected void serviceInit(Configuration conf) throws Exception {
-    this.configuration = conf;
+    timelineServiceVersion =
+        conf.getFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION,
+            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_VERSION);
+    LOG.info("Timeline service address: " + getTimelineServiceAddress());
+    if (!YarnConfiguration.timelineServiceEnabled(conf)
+        || !((Float.compare(this.timelineServiceVersion, 1.0f) == 0)
+            || (Float.compare(this.timelineServiceVersion, 1.5f) == 0))) {
+      throw new IOException("Timeline V1 client is not properly configured. "
+          + "Either timeline service is not enabled or version is not set to"
+          + " 1.x");
+    }
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
     UserGroupInformation realUgi = ugi.getRealUser();
     if (realUgi != null) {
@@ -299,62 +116,34 @@ public class TimelineClientImpl extends TimelineClient {
       authUgi = ugi;
       doAsUser = null;
     }
-    ClientConfig cc = new DefaultClientConfig();
-    cc.getClasses().add(YarnJacksonJaxbJsonProvider.class);
-    connConfigurator = initConnConfigurator(conf);
-    if (UserGroupInformation.isSecurityEnabled()) {
-      authenticator = new KerberosDelegationTokenAuthenticator();
-    } else {
-      authenticator = new PseudoDelegationTokenAuthenticator();
-    }
-    authenticator.setConnectionConfigurator(connConfigurator);
     token = new DelegationTokenAuthenticatedURL.Token();
+    connector = createTimelineConnector();
 
-    connectionRetry = new TimelineClientConnectionRetry(conf);
-    client = new Client(new URLConnectionClientHandler(
-        new TimelineURLConnectionFactory()), cc);
-    TimelineJerseyRetryFilter retryFilter = new TimelineJerseyRetryFilter();
-    // TODO need to cleanup filter retry later.
-    if (!timelineServiceV2) {
-      client.addFilter(retryFilter);
-    }
-
-    // old version timeline service need to get address from configuration
-    // while new version need to auto discovery (with retry).
-    if (timelineServiceV2) {
-      maxServiceRetries = conf.getInt(
-          YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
-          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES);
-      serviceRetryInterval = conf.getLong(
-          YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
-          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS);
-      entityDispatcher = new TimelineEntityDispatcher(conf);
+    if (YarnConfiguration.useHttps(conf)) {
+      timelineServiceAddress =
+          conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
+              YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS);
     } else {
-      if (YarnConfiguration.useHttps(conf)) {
-        setTimelineServiceAddress(conf.get(
-            YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
-            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS));
-      } else {
-        setTimelineServiceAddress(conf.get(
-            YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
-            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS));
-      }
-      timelineServiceVersion =
-          conf.getFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION,
-              YarnConfiguration.DEFAULT_TIMELINE_SERVICE_VERSION);
-      LOG.info("Timeline service address: " + getTimelineServiceAddress());
+      timelineServiceAddress =
+          conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+              YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS);
     }
     super.serviceInit(conf);
   }
 
+  @VisibleForTesting
+  protected TimelineConnector createTimelineConnector() {
+    TimelineConnector newConnector =
+        new TimelineConnector(true, authUgi, doAsUser, token);
+    addIfService(newConnector);
+    return newConnector;
+  }
+
   @Override
   protected void serviceStart() throws Exception {
-    if (timelineServiceV2) {
-      entityDispatcher.start();
-    } else {
-      timelineWriter = createTimelineWriter(configuration, authUgi, client,
-          constructResURI(getConfig(), timelineServiceAddress, false));
-    }
+    timelineWriter = createTimelineWriter(getConfig(), authUgi,
+        connector.getClient(), TimelineConnector.constructResURI(getConfig(),
+            timelineServiceAddress, RESOURCE_URI_STR_V1));
   }
 
   protected TimelineWriter createTimelineWriter(Configuration conf,
@@ -373,12 +162,6 @@ public class TimelineClientImpl extends TimelineClient {
     if (this.timelineWriter != null) {
       this.timelineWriter.close();
     }
-    if (timelineServiceV2) {
-      entityDispatcher.stop();
-    }
-    if (this.sslFactory != null) {
-      this.sslFactory.destroy();
-    }
     super.serviceStop();
   }
 
@@ -390,132 +173,17 @@ public class TimelineClientImpl extends TimelineClient {
   }
 
   @Override
-  public TimelinePutResponse putEntities(
-      TimelineEntity... entities) throws IOException, YarnException {
+  public TimelinePutResponse putEntities(TimelineEntity... entities)
+      throws IOException, YarnException {
     return timelineWriter.putEntities(entities);
   }
 
   @Override
-  public void putEntities(
-      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity...
-          entities) throws IOException, YarnException {
-    if (!timelineServiceV2) {
-      throw new YarnException("v.2 method is invoked on a v.1.x client");
-    }
-    entityDispatcher.dispatchEntities(true, entities);
-  }
-
-  @Override
-  public void putEntitiesAsync(
-      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity...
-          entities) throws IOException, YarnException {
-    if (!timelineServiceV2) {
-      throw new YarnException("v.2 method is invoked on a v.1.x client");
-    }
-    entityDispatcher.dispatchEntities(false, entities);
-  }
-
-  @Override
   public void putDomain(TimelineDomain domain) throws IOException,
       YarnException {
     timelineWriter.putDomain(domain);
   }
 
-  // Used for new timeline service only
-  @Private
-  protected void putObjects(String path, MultivaluedMap<String, String> params,
-      Object obj) throws IOException, YarnException {
-
-    int retries = verifyRestEndPointAvailable();
-
-    // timelineServiceAddress could be stale, add retry logic here.
-    boolean needRetry = true;
-    while (needRetry) {
-      try {
-        URI uri = constructResURI(getConfig(), timelineServiceAddress, true);
-        putObjects(uri, path, params, obj);
-        needRetry = false;
-      } catch (IOException e) {
-        // handle exception for timelineServiceAddress being updated.
-        checkRetryWithSleep(retries, e);
-        retries--;
-      }
-    }
-  }
-
-  private int verifyRestEndPointAvailable() throws YarnException {
-    // timelineServiceAddress could haven't be initialized yet
-    // or stale (only for new timeline service)
-    int retries = pollTimelineServiceAddress(this.maxServiceRetries);
-    if (timelineServiceAddress == null) {
-      String errMessage = "TimelineClient has reached to max retry times : "
-          + this.maxServiceRetries
-          + ", but failed to fetch timeline service address. Please verify"
-          + " Timeline Auxiliary Service is configured in all the NMs";
-      LOG.error(errMessage);
-      throw new YarnException(errMessage);
-    }
-    return retries;
-  }
-
-  /**
-   * Check if reaching to maximum of retries.
-   * @param retries
-   * @param e
-   */
-  private void checkRetryWithSleep(int retries, IOException e)
-      throws YarnException, IOException {
-    if (retries > 0) {
-      try {
-        Thread.sleep(this.serviceRetryInterval);
-      } catch (InterruptedException ex) {
-        Thread.currentThread().interrupt();
-        throw new YarnException("Interrupted while retrying to connect to ATS");
-      }
-    } else {
-      StringBuilder msg =
-          new StringBuilder("TimelineClient has reached to max retry times : ");
-      msg.append(this.maxServiceRetries);
-      msg.append(" for service address: ");
-      msg.append(timelineServiceAddress);
-      LOG.error(msg.toString());
-      throw new IOException(msg.toString(), e);
-    }
-  }
-
-  protected void putObjects(
-      URI base, String path, MultivaluedMap<String, String> params, Object obj)
-          throws IOException, YarnException {
-    ClientResponse resp;
-    try {
-      resp = client.resource(base).path(path).queryParams(params)
-          .accept(MediaType.APPLICATION_JSON)
-          .type(MediaType.APPLICATION_JSON)
-          .put(ClientResponse.class, obj);
-    } catch (RuntimeException re) {
-      // runtime exception is expected if the client cannot connect the server
-      String msg =
-          "Failed to get the response from the timeline server.";
-      LOG.error(msg, re);
-      throw new IOException(re);
-    }
-    if (resp == null ||
-        resp.getStatusInfo().getStatusCode() !=
-            ClientResponse.Status.OK.getStatusCode()) {
-      String msg = "Response from the timeline server is " +
-          ((resp == null) ? "null":
-          "not successful," + " HTTP error code: " + resp.getStatus()
-          + ", Server response:\n" + resp.getEntity(String.class));
-      LOG.error(msg);
-      throw new YarnException(msg);
-    }
-  }
-
-  @Override
-  public void setTimelineServiceAddress(String address) {
-    this.timelineServiceAddress = address;
-  }
-
   private String getTimelineServiceAddress() {
     return this.timelineServiceAddress;
   }
@@ -532,17 +200,17 @@ public class TimelineClientImpl extends TimelineClient {
           public Token<TimelineDelegationTokenIdentifier> run()
               throws Exception {
             DelegationTokenAuthenticatedURL authUrl =
-                new DelegationTokenAuthenticatedURL(authenticator,
-                    connConfigurator);
+                connector.getDelegationTokenAuthenticatedURL();
             // TODO we should add retry logic here if timelineServiceAddress is
             // not available immediately.
             return (Token) authUrl.getDelegationToken(
-                constructResURI(getConfig(),
-                    getTimelineServiceAddress(), false).toURL(),
+                TimelineConnector.constructResURI(getConfig(),
+                    getTimelineServiceAddress(), RESOURCE_URI_STR_V1).toURL(),
                 token, renewer, doAsUser);
           }
         };
-    return (Token<TimelineDelegationTokenIdentifier>) operateDelegationToken(getDTAction);
+    return (Token<TimelineDelegationTokenIdentifier>) connector
+        .operateDelegationToken(getDTAction);
   }
 
   @SuppressWarnings("unchecked")
@@ -568,26 +236,26 @@ public class TimelineClientImpl extends TimelineClient {
               token.setDelegationToken((Token) timelineDT);
             }
             DelegationTokenAuthenticatedURL authUrl =
-                new DelegationTokenAuthenticatedURL(authenticator,
-                    connConfigurator);
+                connector.getDelegationTokenAuthenticatedURL();
             // If the token service address is not available, fall back to use
             // the configured service address.
-            final URI serviceURI = isTokenServiceAddrEmpty ?
-                constructResURI(getConfig(), getTimelineServiceAddress(), false)
+            final URI serviceURI = isTokenServiceAddrEmpty
+                ? TimelineConnector.constructResURI(getConfig(),
+                    getTimelineServiceAddress(), RESOURCE_URI_STR_V1)
                 : new URI(scheme, null, address.getHostName(),
-                address.getPort(), RESOURCE_URI_STR_V1, null, null);
+                    address.getPort(), RESOURCE_URI_STR_V1, null, null);
             return authUrl
                 .renewDelegationToken(serviceURI.toURL(), token, doAsUser);
           }
         };
-    return (Long) operateDelegationToken(renewDTAction);
+    return (Long) connector.operateDelegationToken(renewDTAction);
   }
 
   @SuppressWarnings("unchecked")
   @Override
   public void cancelDelegationToken(
       final Token<TimelineDelegationTokenIdentifier> timelineDT)
-          throws IOException, YarnException {
+      throws IOException, YarnException {
     final boolean isTokenServiceAddrEmpty =
         timelineDT.getService().toString().isEmpty();
     final String scheme = isTokenServiceAddrEmpty ? null
@@ -607,134 +275,29 @@ public class TimelineClientImpl extends TimelineClient {
               token.setDelegationToken((Token) timelineDT);
             }
             DelegationTokenAuthenticatedURL authUrl =
-                new DelegationTokenAuthenticatedURL(authenticator,
-                    connConfigurator);
+                connector.getDelegationTokenAuthenticatedURL();
             // If the token service address is not available, fall back to use
             // the configured service address.
-            final URI serviceURI = isTokenServiceAddrEmpty ?
-                constructResURI(getConfig(), getTimelineServiceAddress(), false)
+            final URI serviceURI = isTokenServiceAddrEmpty
+                ? TimelineConnector.constructResURI(getConfig(),
+                    getTimelineServiceAddress(), RESOURCE_URI_STR_V1)
                 : new URI(scheme, null, address.getHostName(),
-                address.getPort(), RESOURCE_URI_STR_V1, null, null);
+                    address.getPort(), RESOURCE_URI_STR_V1, null, null);
             authUrl.cancelDelegationToken(serviceURI.toURL(), token, doAsUser);
             return null;
           }
         };
-    operateDelegationToken(cancelDTAction);
+    connector.operateDelegationToken(cancelDTAction);
   }
 
   @Override
   public String toString() {
     return super.toString() + " with timeline server "
-        + constructResURI(getConfig(), getTimelineServiceAddress(), false)
+        + TimelineConnector.constructResURI(getConfig(),
+            getTimelineServiceAddress(), RESOURCE_URI_STR_V1)
         + " and writer " + timelineWriter;
   }
 
-  private Object operateDelegationToken(
-      final PrivilegedExceptionAction<?> action)
-      throws IOException, YarnException {
-    // Set up the retry operation
-    TimelineClientRetryOp tokenRetryOp =
-        createTimelineClientRetryOpForOperateDelegationToken(action);
-
-    return connectionRetry.retryOn(tokenRetryOp);
-  }
-
-  /**
-   * Poll TimelineServiceAddress for maximum of retries times if it is null.
-   *
-   * @param retries
-   * @return the left retry times
-   * @throws IOException
-   */
-  private int pollTimelineServiceAddress(int retries) throws YarnException {
-    while (timelineServiceAddress == null && retries > 0) {
-      try {
-        Thread.sleep(this.serviceRetryInterval);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        throw new YarnException("Interrupted while trying to connect ATS");
-      }
-      retries--;
-    }
-    return retries;
-  }
-
-  private class TimelineURLConnectionFactory
-      implements HttpURLConnectionFactory {
-
-    @Override
-    public HttpURLConnection getHttpURLConnection(final URL url) throws IOException {
-      authUgi.checkTGTAndReloginFromKeytab();
-      try {
-        return new DelegationTokenAuthenticatedURL(
-            authenticator, connConfigurator).openConnection(url, token,
-              doAsUser);
-      } catch (UndeclaredThrowableException e) {
-        throw new IOException(e.getCause());
-      } catch (AuthenticationException ae) {
-        throw new IOException(ae);
-      }
-    }
-
-  }
-
-  private ConnectionConfigurator initConnConfigurator(Configuration conf) {
-    try {
-      return initSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, conf);
-    } catch (Exception e) {
-      LOG.debug("Cannot load customized ssl related configuration. " +
-          "Fallback to system-generic settings.", e);
-      return DEFAULT_TIMEOUT_CONN_CONFIGURATOR;
-    }
-  }
-
-  private static final ConnectionConfigurator DEFAULT_TIMEOUT_CONN_CONFIGURATOR =
-      new ConnectionConfigurator() {
-    @Override
-    public HttpURLConnection configure(HttpURLConnection conn)
-        throws IOException {
-      setTimeouts(conn, DEFAULT_SOCKET_TIMEOUT);
-      return conn;
-    }
-  };
-
-  private ConnectionConfigurator initSslConnConfigurator(final int timeout,
-      Configuration conf) throws IOException, GeneralSecurityException {
-    final SSLSocketFactory sf;
-    final HostnameVerifier hv;
-
-    sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
-    sslFactory.init();
-    sf = sslFactory.createSSLSocketFactory();
-    hv = sslFactory.getHostnameVerifier();
-
-    return new ConnectionConfigurator() {
-      @Override
-      public HttpURLConnection configure(HttpURLConnection conn)
-          throws IOException {
-        if (conn instanceof HttpsURLConnection) {
-          HttpsURLConnection c = (HttpsURLConnection) conn;
-          c.setSSLSocketFactory(sf);
-          c.setHostnameVerifier(hv);
-        }
-        setTimeouts(conn, timeout);
-        return conn;
-      }
-    };
-  }
-
-  private static void setTimeouts(URLConnection connection, int socketTimeout) {
-    connection.setConnectTimeout(socketTimeout);
-    connection.setReadTimeout(socketTimeout);
-  }
-
-  private static URI constructResURI(
-      Configuration conf, String address, boolean v2) {
-    return URI.create(
-        JOINER.join(YarnConfiguration.useHttps(conf) ? "https://" : "http://",
-            address, v2 ? RESOURCE_URI_STR_V2 : RESOURCE_URI_STR_V1));
-  }
-
   public static void main(String[] argv) throws Exception {
     CommandLine cliParser = new GnuParser().parse(opts, argv);
     if (cliParser.hasOption("put")) {
@@ -870,266 +433,4 @@ public class TimelineClientImpl extends TimelineClient {
   public void setTimelineWriter(TimelineWriter writer) {
     this.timelineWriter = writer;
   }
-
-  @Private
-  @VisibleForTesting
-  public TimelineClientRetryOp
-      createTimelineClientRetryOpForOperateDelegationToken(
-          final PrivilegedExceptionAction<?> action) throws IOException {
-    return new TimelineClientRetryOpForOperateDelegationToken(
-        this.authUgi, action);
-  }
-
-  @Private
-  @VisibleForTesting
-  public class TimelineClientRetryOpForOperateDelegationToken
-      extends TimelineClientRetryOp {
-
-    private final UserGroupInformation authUgi;
-    private final PrivilegedExceptionAction<?> action;
-
-    public TimelineClientRetryOpForOperateDelegationToken(
-        UserGroupInformation authUgi, PrivilegedExceptionAction<?> action) {
-      this.authUgi = authUgi;
-      this.action = action;
-    }
-
-    @Override
-    public Object run() throws IOException {
-      // Try pass the request, if fail, keep retrying
-      authUgi.checkTGTAndReloginFromKeytab();
-      try {
-        return authUgi.doAs(action);
-      } catch (UndeclaredThrowableException e) {
-        throw new IOException(e.getCause());
-      } catch (InterruptedException e) {
-        throw new IOException(e);
-      }
-    }
-
-    @Override
-    public boolean shouldRetryOn(Exception e) {
-      // retry on connection exceptions
-      // and SocketTimeoutException
-      return (e instanceof ConnectException
-          || e instanceof SocketTimeoutException);
-    }
-  }
-
-  private final class EntitiesHolder extends FutureTask<Void> {
-    private final
-        org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
-            entities;
-    private final boolean isSync;
-
-    EntitiesHolder(
-        final
-            org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
-                entities,
-        final boolean isSync) {
-      super(new Callable<Void>() {
-        // publishEntities()
-        public Void call() throws Exception {
-          MultivaluedMap<String, String> params = new MultivaluedMapImpl();
-          params.add("appid", getContextAppId().toString());
-          params.add("async", Boolean.toString(!isSync));
-          putObjects("entities", params, entities);
-          return null;
-        }
-      });
-      this.entities = entities;
-      this.isSync = isSync;
-    }
-
-    public boolean isSync() {
-      return isSync;
-    }
-
-    public org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
-        getEntities() {
-      return entities;
-    }
-  }
-
-  /**
-   * This class is responsible for collecting the timeline entities and
-   * publishing them in async.
-   */
-  private class TimelineEntityDispatcher {
-    /**
-     * Time period for which the timelineclient will wait for draining after
-     * stop.
-     */
-    private static final long DRAIN_TIME_PERIOD = 2000L;
-
-    private int numberOfAsyncsToMerge;
-    private final BlockingQueue<EntitiesHolder> timelineEntityQueue;
-    private ExecutorService executor;
-
-    TimelineEntityDispatcher(Configuration conf) {
-      timelineEntityQueue = new LinkedBlockingQueue<EntitiesHolder>();
-      numberOfAsyncsToMerge =
-          conf.getInt(YarnConfiguration.NUMBER_OF_ASYNC_ENTITIES_TO_MERGE,
-              YarnConfiguration.DEFAULT_NUMBER_OF_ASYNC_ENTITIES_TO_MERGE);
-    }
-
-    Runnable createRunnable() {
-      return new Runnable() {
-        @Override
-        public void run() {
-          try {
-            EntitiesHolder entitiesHolder;
-            while (!Thread.currentThread().isInterrupted()) {
-              // Merge all the async calls and make one push, but if its sync
-              // call push immediately
-              try {
-                entitiesHolder = timelineEntityQueue.take();
-              } catch (InterruptedException ie) {
-                LOG.info("Timeline dispatcher thread was interrupted ");
-                Thread.currentThread().interrupt();
-                return;
-              }
-              if (entitiesHolder != null) {
-                publishWithoutBlockingOnQueue(entitiesHolder);
-              }
-            }
-          } finally {
-            if (!timelineEntityQueue.isEmpty()) {
-              LOG.info("Yet to publish " + timelineEntityQueue.size()
-                  + " timelineEntities, draining them now. ");
-            }
-            // Try to drain the remaining entities to be published @ the max for
-            // 2 seconds
-            long timeTillweDrain =
-                System.currentTimeMillis() + DRAIN_TIME_PERIOD;
-            while (!timelineEntityQueue.isEmpty()) {
-              publishWithoutBlockingOnQueue(timelineEntityQueue.poll());
-              if (System.currentTimeMillis() > timeTillweDrain) {
-                // time elapsed stop publishing further....
-                if (!timelineEntityQueue.isEmpty()) {
-                  LOG.warn("Time to drain elapsed! Remaining "
-                      + timelineEntityQueue.size() + "timelineEntities will not"
-                      + " be published");
-                  // if some entities were not drained then we need interrupt
-                  // the threads which had put sync EntityHolders to the queue.
-                  EntitiesHolder nextEntityInTheQueue = null;
-                  while ((nextEntityInTheQueue =
-                      timelineEntityQueue.poll()) != null) {
-                    nextEntityInTheQueue.cancel(true);
-                  }
-                }
-                break;
-              }
-            }
-          }
-        }
-
-        /**
-         * Publishes the given EntitiesHolder and return immediately if sync
-         * call, else tries to fetch the EntitiesHolder from the queue in non
-         * blocking fashion and collate the Entities if possible before
-         * publishing through REST.
-         *
-         * @param entitiesHolder
-         */
-        private void publishWithoutBlockingOnQueue(
-            EntitiesHolder entitiesHolder) {
-          if (entitiesHolder.isSync()) {
-            entitiesHolder.run();
-            return;
-          }
-          int count = 1;
-          while (true) {
-            // loop till we find a sync put Entities or there is nothing
-            // to take
-            EntitiesHolder nextEntityInTheQueue = timelineEntityQueue.poll();
-            if (nextEntityInTheQueue == null) {
-              // Nothing in the queue just publish and get back to the
-              // blocked wait state
-              entitiesHolder.run();
-              break;
-            } else if (nextEntityInTheQueue.isSync()) {
-              // flush all the prev async entities first
-              entitiesHolder.run();
-              // and then flush the sync entity
-              nextEntityInTheQueue.run();
-              break;
-            } else {
-              // append all async entities together and then flush
-              entitiesHolder.getEntities().addEntities(
-                  nextEntityInTheQueue.getEntities().getEntities());
-              count++;
-              if (count == numberOfAsyncsToMerge) {
-                // Flush the entities if the number of the async
-                // putEntites merged reaches the desired limit. To avoid
-                // collecting multiple entities and delaying for a long
-                // time.
-                entitiesHolder.run();
-                break;
-              }
-            }
-          }
-        }
-      };
-    }
-
-    public void dispatchEntities(boolean sync,
-        org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity[]
-            entitiesTobePublished) throws YarnException {
-      if (executor.isShutdown()) {
-        throw new YarnException("Timeline client is in the process of stopping,"
-            + " not accepting any more TimelineEntities");
-      }
-
-      // wrap all TimelineEntity into TimelineEntities object
-      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
-          entities =
-              new org.apache.hadoop.yarn.api.records.timelineservice.
-                  TimelineEntities();
-      for (org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
-               entity : entitiesTobePublished) {
-        entities.addEntity(entity);
-      }
-
-      // created a holder and place it in queue
-      EntitiesHolder entitiesHolder = new EntitiesHolder(entities, sync);
-      try {
-        timelineEntityQueue.put(entitiesHolder);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        throw new YarnException(
-            "Failed while adding entity to the queue for publishing", e);
-      }
-
-      if (sync) {
-        // In sync call we need to wait till its published and if any error then
-        // throw it back
-        try {
-          entitiesHolder.get();
-        } catch (ExecutionException e) {
-          throw new YarnException("Failed while publishing entity",
-              e.getCause());
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          throw new YarnException("Interrupted while publishing entity", e);
-        }
-      }
-    }
-
-    public void start() {
-      executor = Executors.newSingleThreadExecutor();
-      executor.execute(createRunnable());
-    }
-
-    public void stop() {
-      LOG.info("Stopping TimelineClient.");
-      executor.shutdownNow();
-      try {
-        executor.awaitTermination(DRAIN_TIME_PERIOD, TimeUnit.MILLISECONDS);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        e.printStackTrace();
-      }
-    }
-  }
 }


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