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 ae...@apache.org on 2017/12/07 00:53:46 UTC

[01/50] [abbrv] hadoop git commit: YARN-7541. Node updates don't update the maximum cluster capability for resources other than CPU and memory

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 15cce4495 -> 5f16cdbd2


YARN-7541. Node updates don't update the maximum cluster capability for resources other than CPU and memory


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

Branch: refs/heads/HDFS-7240
Commit: 8498d287cd3beddcf8fe19625227e09982ec4be2
Parents: 3016418
Author: Daniel Templeton <te...@apache.org>
Authored: Wed Nov 29 10:36:19 2017 -0800
Committer: Daniel Templeton <te...@apache.org>
Committed: Wed Nov 29 11:11:36 2017 -0800

----------------------------------------------------------------------
 .../hadoop/yarn/api/records/Resource.java       |  24 +++-
 .../yarn/util/resource/ResourceUtils.java       |  29 +++++
 .../scheduler/ClusterNodeTracker.java           |  79 ++++++++----
 .../yarn/server/resourcemanager/MockNodes.java  |   4 +
 .../scheduler/TestClusterNodeTracker.java       | 125 ++++++++++++++++++-
 5 files changed, 229 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8498d287/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
index abd44b8..b32955b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
@@ -21,11 +21,9 @@ package org.apache.hadoop.yarn.api.records;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Map;
 
-import com.google.common.collect.Lists;
-import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.lang.NotImplementedException;
-import org.apache.curator.shaded.com.google.common.reflect.ClassPath;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
@@ -89,6 +87,26 @@ public abstract class Resource implements Comparable<Resource> {
     return new LightWeightResource(memory, vCores);
   }
 
+  /**
+   * Create a new {@link Resource} instance with the given CPU and memory
+   * values and additional resource values as set in the {@code others}
+   * parameter. Note that the CPU and memory settings in the {@code others}
+   * parameter will be ignored.
+   *
+   * @param memory the memory value
+   * @param vCores the CPU value
+   * @param others a map of other resource values indexed by resource name
+   * @return a {@link Resource} instance with the given resource values
+   */
+  @Public
+  @Stable
+  public static Resource newInstance(long memory, int vCores,
+      Map<String, Long> others) {
+    ResourceInformation[] info = ResourceUtils.createResourceTypesArray(others);
+
+    return new LightWeightResource(memory, vCores, info);
+  }
+
   @InterfaceAudience.Private
   @InterfaceStability.Unstable
   public static Resource newInstance(Resource resource) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8498d287/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
index c168337..3c6ca98 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
@@ -629,4 +629,33 @@ public class ResourceUtils {
     return result;
   }
 
+  /**
+   * Create an array of {@link ResourceInformation} objects corresponding to
+   * the passed in map of names to values. The array will be ordered according
+   * to the order returned by {@link #getResourceTypesArray()}. The value of
+   * each resource type in the returned array will either be the value given for
+   * that resource in the {@code res} parameter or, if none is given, 0.
+   *
+   * @param res the map of resource type values
+   * @return an array of {@link ResourceInformation} instances
+   */
+  public static ResourceInformation[] createResourceTypesArray(Map<String,
+      Long> res) {
+    ResourceInformation[] info = new ResourceInformation[resourceTypes.size()];
+
+    for (Entry<String, Integer> entry : RESOURCE_NAME_TO_INDEX.entrySet()) {
+      int index = entry.getValue();
+      Long value = res.get(entry.getKey());
+
+      if (value == null) {
+        value = 0L;
+      }
+
+      info[index] = new ResourceInformation();
+      ResourceInformation.copy(resourceTypesArray[index], info[index]);
+      info[index].setValue(value);
+    }
+
+    return info;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8498d287/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ClusterNodeTracker.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/ClusterNodeTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ClusterNodeTracker.java
index 9e54ac6..66d8810 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ClusterNodeTracker.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ClusterNodeTracker.java
@@ -24,11 +24,14 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.util.resource.Resources;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -60,11 +63,16 @@ public class ClusterNodeTracker<N extends SchedulerNode> {
       Resources.clone(Resources.none());
 
   // Max allocation
-  private long maxNodeMemory = -1;
-  private int maxNodeVCores = -1;
+  private final long[] maxAllocation;
   private Resource configuredMaxAllocation;
   private boolean forceConfiguredMaxAllocation = true;
   private long configuredMaxAllocationWaitTime;
+  private boolean reportedMaxAllocation = false;
+
+  public ClusterNodeTracker() {
+    maxAllocation = new long[ResourceUtils.getNumberOfKnownResourceTypes()];
+    Arrays.fill(maxAllocation, -1);
+  }
 
   public void addNode(N node) {
     writeLock.lock();
@@ -208,17 +216,18 @@ public class ClusterNodeTracker<N extends SchedulerNode> {
         forceConfiguredMaxAllocation = false;
       }
 
-      if (forceConfiguredMaxAllocation
-          || maxNodeMemory == -1 || maxNodeVCores == -1) {
+      if (forceConfiguredMaxAllocation || !reportedMaxAllocation) {
         return configuredMaxAllocation;
       }
 
       Resource ret = Resources.clone(configuredMaxAllocation);
-      if (ret.getMemorySize() > maxNodeMemory) {
-        ret.setMemorySize(maxNodeMemory);
-      }
-      if (ret.getVirtualCores() > maxNodeVCores) {
-        ret.setVirtualCores(maxNodeVCores);
+
+      for (int i = 0; i < maxAllocation.length; i++) {
+        ResourceInformation info = ret.getResourceInformation(i);
+
+        if (info.getValue() > maxAllocation[i]) {
+          info.setValue(maxAllocation[i]);
+        }
       }
 
       return ret;
@@ -229,31 +238,51 @@ public class ClusterNodeTracker<N extends SchedulerNode> {
 
   private void updateMaxResources(SchedulerNode node, boolean add) {
     Resource totalResource = node.getTotalResource();
+    ResourceInformation[] totalResources;
+
+    if (totalResource != null) {
+      totalResources = totalResource.getResources();
+    } else {
+      LOG.warn(node.getNodeName() + " reported in with null resources, which "
+          + "indicates a problem in the source code. Please file an issue at "
+          + "https://issues.apache.org/jira/secure/CreateIssue!default.jspa");
+
+      return;
+    }
+
     writeLock.lock();
+
     try {
       if (add) { // added node
-        long nodeMemory = totalResource.getMemorySize();
-        if (nodeMemory > maxNodeMemory) {
-          maxNodeMemory = nodeMemory;
-        }
-        int nodeVCores = totalResource.getVirtualCores();
-        if (nodeVCores > maxNodeVCores) {
-          maxNodeVCores = nodeVCores;
+        // If we add a node, we must have a max allocation for all resource
+        // types
+        reportedMaxAllocation = true;
+
+        for (int i = 0; i < maxAllocation.length; i++) {
+          long value = totalResources[i].getValue();
+
+          if (value > maxAllocation[i]) {
+            maxAllocation[i] = value;
+          }
         }
       } else {  // removed node
-        if (maxNodeMemory == totalResource.getMemorySize()) {
-          maxNodeMemory = -1;
-        }
-        if (maxNodeVCores == totalResource.getVirtualCores()) {
-          maxNodeVCores = -1;
+        boolean recalculate = false;
+
+        for (int i = 0; i < maxAllocation.length; i++) {
+          if (totalResources[i].getValue() == maxAllocation[i]) {
+            // No need to set reportedMaxAllocation to false here because we
+            // will recalculate before we release the lock.
+            maxAllocation[i] = -1;
+            recalculate = true;
+          }
         }
+
         // We only have to iterate through the nodes if the current max memory
         // or vcores was equal to the removed node's
-        if (maxNodeMemory == -1 || maxNodeVCores == -1) {
+        if (recalculate) {
           // Treat it like an empty cluster and add nodes
-          for (N n : nodes.values()) {
-            updateMaxResources(n, true);
-          }
+          reportedMaxAllocation = false;
+          nodes.values().forEach(n -> updateMaxResources(n, true));
         }
       }
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8498d287/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
index 611c7f2..317c648 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
@@ -49,6 +49,10 @@ public class MockNodes {
   private static int NODE_ID = 0;
   private static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
 
+  public static void resetHostIds() {
+    NODE_ID = 0;
+  }
+
   public static List<RMNode> newNodes(int racks, int nodesPerRack,
                                         Resource perNode) {
     List<RMNode> list = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8498d287/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestClusterNodeTracker.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/TestClusterNodeTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestClusterNodeTracker.java
index 7f527f1..c1703bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestClusterNodeTracker.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestClusterNodeTracker.java
@@ -17,16 +17,21 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.api.records.Resource;
 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.fair.FSSchedulerNode;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
+
 import org.junit.Before;
 import org.junit.Test;
 
-import java.util.List;
-
 import static org.junit.Assert.assertEquals;
 
 /**
@@ -34,11 +39,15 @@ import static org.junit.Assert.assertEquals;
  * loss of generality.
  */
 public class TestClusterNodeTracker {
-  private ClusterNodeTracker<FSSchedulerNode> nodeTracker =
-      new ClusterNodeTracker<>();
+  private ClusterNodeTracker<FSSchedulerNode> nodeTracker;
 
   @Before
   public void setup() {
+    nodeTracker = new ClusterNodeTracker<>();
+  }
+
+  private void addEight4x4Nodes() {
+    MockNodes.resetHostIds();
     List<RMNode> rmNodes =
         MockNodes.newNodes(2, 4, Resource.newInstance(4096, 4));
     for (RMNode rmNode : rmNodes) {
@@ -48,6 +57,7 @@ public class TestClusterNodeTracker {
 
   @Test
   public void testGetNodeCount() {
+    addEight4x4Nodes();
     assertEquals("Incorrect number of nodes in the cluster",
         8, nodeTracker.nodeCount());
 
@@ -57,6 +67,7 @@ public class TestClusterNodeTracker {
 
   @Test
   public void testGetNodesForResourceName() throws Exception {
+    addEight4x4Nodes();
     assertEquals("Incorrect number of nodes matching ANY",
         8, nodeTracker.getNodesByResourceName(ResourceRequest.ANY).size());
 
@@ -66,4 +77,110 @@ public class TestClusterNodeTracker {
     assertEquals("Incorrect number of nodes matching node",
         1, nodeTracker.getNodesByResourceName("host0").size());
   }
+
+  @Test
+  public void testMaxAllowedAllocation() {
+    // Add a third resource
+    Configuration conf = new Configuration();
+
+    conf.set(YarnConfiguration.RESOURCE_TYPES, "test1");
+
+    ResourceUtils.resetResourceTypes(conf);
+    setup();
+
+    Resource maximum = Resource.newInstance(10240, 10,
+        Collections.singletonMap("test1", 10L));
+
+    nodeTracker.setConfiguredMaxAllocation(maximum);
+
+    Resource result = nodeTracker.getMaxAllowedAllocation();
+
+    assertEquals("With no nodes added, the ClusterNodeTracker did not return "
+        + "the configured max allocation", maximum, result);
+
+    List<RMNode> smallNodes =
+        MockNodes.newNodes(1, 1, Resource.newInstance(1024, 2,
+            Collections.singletonMap("test1", 4L)));
+    FSSchedulerNode smallNode = new FSSchedulerNode(smallNodes.get(0), false);
+    List<RMNode> mediumNodes =
+        MockNodes.newNodes(1, 1, Resource.newInstance(4096, 2,
+            Collections.singletonMap("test1", 2L)));
+    FSSchedulerNode mediumNode = new FSSchedulerNode(mediumNodes.get(0), false);
+    List<RMNode> largeNodes =
+        MockNodes.newNodes(1, 1, Resource.newInstance(16384, 4,
+            Collections.singletonMap("test1", 1L)));
+    FSSchedulerNode largeNode = new FSSchedulerNode(largeNodes.get(0), false);
+
+    nodeTracker.addNode(mediumNode);
+
+    result = nodeTracker.getMaxAllowedAllocation();
+
+    assertEquals("With a single node added, the ClusterNodeTracker did not "
+        + "return that node's resources as the maximum allocation",
+        mediumNodes.get(0).getTotalCapability(), result);
+
+    nodeTracker.addNode(smallNode);
+
+    result = nodeTracker.getMaxAllowedAllocation();
+
+    assertEquals("With two nodes added, the ClusterNodeTracker did not "
+        + "return a the maximum allocation that was the max of their aggregate "
+        + "resources",
+        Resource.newInstance(4096, 2, Collections.singletonMap("test1", 4L)),
+        result);
+
+    nodeTracker.removeNode(smallNode.getNodeID());
+
+    result = nodeTracker.getMaxAllowedAllocation();
+
+    assertEquals("After removing a node, the ClusterNodeTracker did not "
+        + "recalculate the adjusted maximum allocation correctly",
+        mediumNodes.get(0).getTotalCapability(), result);
+
+    nodeTracker.addNode(largeNode);
+
+    result = nodeTracker.getMaxAllowedAllocation();
+
+    assertEquals("With two nodes added, the ClusterNodeTracker did not "
+        + "return a the maximum allocation that was the max of their aggregate "
+        + "resources",
+        Resource.newInstance(10240, 4, Collections.singletonMap("test1", 2L)),
+        result);
+
+    nodeTracker.removeNode(largeNode.getNodeID());
+
+    result = nodeTracker.getMaxAllowedAllocation();
+
+    assertEquals("After removing a node, the ClusterNodeTracker did not "
+        + "recalculate the adjusted maximum allocation correctly",
+        mediumNodes.get(0).getTotalCapability(), result);
+
+    nodeTracker.removeNode(mediumNode.getNodeID());
+
+    result = nodeTracker.getMaxAllowedAllocation();
+
+    assertEquals("After removing all nodes, the ClusterNodeTracker did not "
+        + "return the configured maximum allocation", maximum, result);
+
+    nodeTracker.addNode(smallNode);
+    nodeTracker.addNode(mediumNode);
+    nodeTracker.addNode(largeNode);
+
+    result = nodeTracker.getMaxAllowedAllocation();
+
+    assertEquals("With three nodes added, the ClusterNodeTracker did not "
+        + "return a the maximum allocation that was the max of their aggregate "
+        + "resources",
+        Resource.newInstance(10240, 4, Collections.singletonMap("test1", 4L)),
+        result);
+
+    nodeTracker.removeNode(smallNode.getNodeID());
+    nodeTracker.removeNode(mediumNode.getNodeID());
+    nodeTracker.removeNode(largeNode.getNodeID());
+
+    result = nodeTracker.getMaxAllowedAllocation();
+
+    assertEquals("After removing all nodes, the ClusterNodeTracker did not "
+        + "return the configured maximum allocation", maximum, result);
+  }
 }
\ 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


[31/50] [abbrv] hadoop git commit: HDFS-12396. Webhdfs file system should get delegation token from kms provider. Contributed by Rushabh S Shah.

Posted by ae...@apache.org.
HDFS-12396. Webhdfs file system should get delegation token from kms provider. Contributed by Rushabh S Shah.


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

Branch: refs/heads/HDFS-7240
Commit: 404eab4dc0582e0384b93664ea6ee77ccd5eeebc
Parents: 37ca416
Author: Xiao Chen <xi...@apache.org>
Authored: Mon Dec 4 10:37:48 2017 -0800
Committer: Xiao Chen <xi...@apache.org>
Committed: Mon Dec 4 10:40:11 2017 -0800

----------------------------------------------------------------------
 .../crypto/key/KeyProviderTokenIssuer.java      |  36 ++++
 .../apache/hadoop/crypto/key/package-info.java  |  18 ++
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 109 +----------
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  26 ---
 .../hadoop/hdfs/DistributedFileSystem.java      |  37 ++--
 .../org/apache/hadoop/hdfs/HdfsKMSUtil.java     | 190 +++++++++++++++++++
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  37 +++-
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |   2 +-
 .../apache/hadoop/hdfs/TestEncryptionZones.java |  40 +++-
 .../hadoop/hdfs/TestEncryptionZonesWithKMS.java |  23 +++
 10 files changed, 364 insertions(+), 154 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/404eab4d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderTokenIssuer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderTokenIssuer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderTokenIssuer.java
new file mode 100644
index 0000000..aa5de2c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderTokenIssuer.java
@@ -0,0 +1,36 @@
+/**
+ * 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.crypto.key;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * File systems that support Encryption Zones have to implement this interface.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface KeyProviderTokenIssuer {
+
+  KeyProvider getKeyProvider() throws IOException;
+
+  URI getKeyProviderUri() throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/404eab4d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/package-info.java
new file mode 100644
index 0000000..d66bb44
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/package-info.java
@@ -0,0 +1,18 @@
+/**
+ * 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.crypto.key;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/404eab4d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 3df36d6..51dff8d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_WRITES;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_READAHEAD;
@@ -58,18 +57,15 @@ import javax.net.SocketFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.crypto.CryptoCodec;
 import org.apache.hadoop.crypto.CryptoInputStream;
 import org.apache.hadoop.crypto.CryptoOutputStream;
-import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CacheFlag;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -171,7 +167,6 @@ import org.apache.hadoop.ipc.RpcNoSuchMethodException;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
@@ -207,7 +202,6 @@ import com.google.common.net.InetAddresses;
 public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     DataEncryptionKeyFactory {
   public static final Logger LOG = LoggerFactory.getLogger(DFSClient.class);
-  private static final String DFS_KMS_PREFIX = "dfs-kms-";
 
   private final Configuration conf;
   private final Tracer tracer;
@@ -937,55 +931,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   }
 
   /**
-   * Obtain the crypto protocol version from the provided FileEncryptionInfo,
-   * checking to see if this version is supported by.
-   *
-   * @param feInfo FileEncryptionInfo
-   * @return CryptoProtocolVersion from the feInfo
-   * @throws IOException if the protocol version is unsupported.
-   */
-  private static CryptoProtocolVersion getCryptoProtocolVersion(
-      FileEncryptionInfo feInfo) throws IOException {
-    final CryptoProtocolVersion version = feInfo.getCryptoProtocolVersion();
-    if (!CryptoProtocolVersion.supports(version)) {
-      throw new IOException("Client does not support specified " +
-          "CryptoProtocolVersion " + version.getDescription() + " version " +
-          "number" + version.getVersion());
-    }
-    return version;
-  }
-
-  /**
-   * Obtain a CryptoCodec based on the CipherSuite set in a FileEncryptionInfo
-   * and the available CryptoCodecs configured in the Configuration.
-   *
-   * @param conf   Configuration
-   * @param feInfo FileEncryptionInfo
-   * @return CryptoCodec
-   * @throws IOException if no suitable CryptoCodec for the CipherSuite is
-   *                     available.
-   */
-  private static CryptoCodec getCryptoCodec(Configuration conf,
-      FileEncryptionInfo feInfo) throws IOException {
-    final CipherSuite suite = feInfo.getCipherSuite();
-    if (suite.equals(CipherSuite.UNKNOWN)) {
-      throw new IOException("NameNode specified unknown CipherSuite with ID "
-          + suite.getUnknownValue() + ", cannot instantiate CryptoCodec.");
-    }
-    final CryptoCodec codec = CryptoCodec.getInstance(conf, suite);
-    if (codec == null) {
-      throw new UnknownCipherSuiteException(
-          "No configuration found for the cipher suite "
-              + suite.getConfigSuffix() + " prefixed with "
-              + HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
-              + ". Please see the example configuration "
-              + "hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE "
-              + "at core-default.xml for details.");
-    }
-    return codec;
-  }
-
-  /**
    * Wraps the stream in a CryptoInputStream if the underlying file is
    * encrypted.
    */
@@ -995,8 +940,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     if (feInfo != null) {
       // File is encrypted, wrap the stream in a crypto stream.
       // Currently only one version, so no special logic based on the version #
-      getCryptoProtocolVersion(feInfo);
-      final CryptoCodec codec = getCryptoCodec(conf, feInfo);
+      HdfsKMSUtil.getCryptoProtocolVersion(feInfo);
+      final CryptoCodec codec = HdfsKMSUtil.getCryptoCodec(conf, feInfo);
       final KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
       final CryptoInputStream cryptoIn =
           new CryptoInputStream(dfsis, codec, decrypted.getMaterial(),
@@ -1027,8 +972,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     if (feInfo != null) {
       // File is encrypted, wrap the stream in a crypto stream.
       // Currently only one version, so no special logic based on the version #
-      getCryptoProtocolVersion(feInfo);
-      final CryptoCodec codec = getCryptoCodec(conf, feInfo);
+      HdfsKMSUtil.getCryptoProtocolVersion(feInfo);
+      final CryptoCodec codec = HdfsKMSUtil.getCryptoCodec(conf, feInfo);
       KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
       final CryptoOutputStream cryptoOut =
           new CryptoOutputStream(dfsos, codec,
@@ -2983,51 +2928,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     return HEDGED_READ_METRIC;
   }
 
-  /**
-   * Returns a key to map namenode uri to key provider uri.
-   * Tasks will lookup this key to find key Provider.
-   */
-  public Text getKeyProviderMapKey() {
-    return new Text(DFS_KMS_PREFIX + namenodeUri.getScheme()
-        +"://" + namenodeUri.getAuthority());
-  }
-
-  /**
-   * The key provider uri is searched in the following order.
-   * 1. If there is a mapping in Credential's secrets map for namenode uri.
-   * 2. From namenode getServerDefaults rpc.
-   * 3. Finally fallback to local conf.
-   * @return keyProviderUri if found from either of above 3 cases,
-   * null otherwise
-   * @throws IOException
-   */
   URI getKeyProviderUri() throws IOException {
-    URI keyProviderUri = null;
-    // Lookup the secret in credentials object for namenodeuri.
-    Credentials credentials = ugi.getCredentials();
-    byte[] keyProviderUriBytes = credentials.getSecretKey(getKeyProviderMapKey());
-    if(keyProviderUriBytes != null) {
-      keyProviderUri =
-          URI.create(DFSUtilClient.bytes2String(keyProviderUriBytes));
-      return keyProviderUri;
-    }
-
-    // Query the namenode for the key provider uri.
-    FsServerDefaults serverDefaults = getServerDefaults();
-    if (serverDefaults.getKeyProviderUri() != null) {
-      if (!serverDefaults.getKeyProviderUri().isEmpty()) {
-        keyProviderUri = URI.create(serverDefaults.getKeyProviderUri());
-      }
-      return keyProviderUri;
-    }
-
-    // Last thing is to trust its own conf to be backwards compatible.
-    String keyProviderUriStr = conf.getTrimmed(
-        CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH);
-    if (keyProviderUriStr != null && !keyProviderUriStr.isEmpty()) {
-      keyProviderUri = URI.create(keyProviderUriStr);
-    }
-    return keyProviderUri;
+    return HdfsKMSUtil.getKeyProviderUri(ugi, namenodeUri,
+        getServerDefaults().getKeyProviderUri(), conf);
   }
 
   public KeyProvider getKeyProvider() throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/404eab4d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
index f6b28e0..32e5d0f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
@@ -23,7 +23,6 @@ import com.google.common.collect.Maps;
 import com.google.common.primitives.SignedBytes;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
@@ -54,7 +53,6 @@ import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Daemon;
-import org.apache.hadoop.util.KMSUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -596,30 +594,6 @@ public class DFSUtilClient {
     return new ReconfigurationProtocolTranslatorPB(addr, ticket, conf, factory);
   }
 
-  private static String keyProviderUriKeyName =
-      CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH;
-
-  /**
-   * Set the key provider uri configuration key name for creating key providers.
-   * @param keyName The configuration key name.
-   */
-  public static void setKeyProviderUriKeyName(final String keyName) {
-    keyProviderUriKeyName = keyName;
-  }
-
-  /**
-   * Creates a new KeyProvider from the given Configuration.
-   *
-   * @param conf Configuration
-   * @return new KeyProvider, or null if no provider was found.
-   * @throws IOException if the KeyProvider is improperly specified in
-   *                             the Configuration
-   */
-  public static KeyProvider createKeyProvider(
-      final Configuration conf) throws IOException {
-    return KMSUtil.createKeyProvider(conf, keyProviderUriKeyName);
-  }
-
   public static Peer peerFromSocket(Socket socket)
       throws IOException {
     Peer peer;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/404eab4d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index c010c8a..1478868 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -26,7 +26,8 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.KeyProviderTokenIssuer;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockStoragePolicySpi;
 import org.apache.hadoop.fs.CacheFlag;
@@ -124,7 +125,8 @@ import java.util.stream.Collectors;
  *****************************************************************/
 @InterfaceAudience.LimitedPrivate({ "MapReduce", "HBase" })
 @InterfaceStability.Unstable
-public class DistributedFileSystem extends FileSystem {
+public class DistributedFileSystem extends FileSystem
+    implements KeyProviderTokenIssuer {
   private Path workingDir;
   private URI uri;
   private String homeDirPrefix =
@@ -2605,28 +2607,21 @@ public class DistributedFileSystem extends FileSystem {
   }
 
   @Override
+  public URI getKeyProviderUri() throws IOException {
+    return dfs.getKeyProviderUri();
+  }
+
+  @Override
+  public KeyProvider getKeyProvider() throws IOException {
+    return dfs.getKeyProvider();
+  }
+
+  @Override
   public Token<?>[] addDelegationTokens(
       final String renewer, Credentials credentials) throws IOException {
     Token<?>[] tokens = super.addDelegationTokens(renewer, credentials);
-    URI keyProviderUri = dfs.getKeyProviderUri();
-    if (keyProviderUri != null) {
-      KeyProviderDelegationTokenExtension keyProviderDelegationTokenExtension =
-          KeyProviderDelegationTokenExtension.
-              createKeyProviderDelegationTokenExtension(dfs.getKeyProvider());
-      Token<?>[] kpTokens = keyProviderDelegationTokenExtension.
-          addDelegationTokens(renewer, credentials);
-      credentials.addSecretKey(dfs.getKeyProviderMapKey(),
-          DFSUtilClient.string2Bytes(keyProviderUri.toString()));
-      if (tokens != null && kpTokens != null) {
-        Token<?>[] all = new Token<?>[tokens.length + kpTokens.length];
-        System.arraycopy(tokens, 0, all, 0, tokens.length);
-        System.arraycopy(kpTokens, 0, all, tokens.length, kpTokens.length);
-        tokens = all;
-      } else {
-        tokens = (tokens != null) ? tokens : kpTokens;
-      }
-    }
-    return tokens;
+    return HdfsKMSUtil.addDelegationTokensForKeyProvider(
+        this, renewer, credentials, uri, tokens);
   }
 
   public DFSInotifyEventInputStream getInotifyEventStream() throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/404eab4d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsKMSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsKMSUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsKMSUtil.java
new file mode 100644
index 0000000..71d2972
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsKMSUtil.java
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CipherSuite;
+import org.apache.hadoop.crypto.CryptoCodec;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
+import org.apache.hadoop.crypto.key.KeyProviderTokenIssuer;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.KMSUtil;
+
+/**
+ * Utility class for key provider related methods in hdfs client package.
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class HdfsKMSUtil {
+  private static final String DFS_KMS_PREFIX = "dfs-kms-";
+  private static String keyProviderUriKeyName =
+      CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH;
+
+  private HdfsKMSUtil() { /* Hidden constructor */ }
+
+  /**
+   * Creates a new KeyProvider from the given Configuration.
+   *
+   * @param conf Configuration
+   * @return new KeyProvider, or null if no provider was found.
+   * @throws IOException if the KeyProvider is improperly specified in
+   *                             the Configuration
+   */
+  public static KeyProvider createKeyProvider(
+      final Configuration conf) throws IOException {
+    return KMSUtil.createKeyProvider(conf, keyProviderUriKeyName);
+  }
+
+  public static Token<?>[] addDelegationTokensForKeyProvider(
+      KeyProviderTokenIssuer kpTokenIssuer, final String renewer,
+      Credentials credentials, URI namenodeUri, Token<?>[] tokens)
+          throws IOException {
+    KeyProvider keyProvider = kpTokenIssuer.getKeyProvider();
+    if (keyProvider != null) {
+      KeyProviderDelegationTokenExtension keyProviderDelegationTokenExtension
+          = KeyProviderDelegationTokenExtension.
+              createKeyProviderDelegationTokenExtension(keyProvider);
+      Token<?>[] kpTokens = keyProviderDelegationTokenExtension.
+          addDelegationTokens(renewer, credentials);
+      credentials.addSecretKey(getKeyProviderMapKey(namenodeUri),
+          DFSUtilClient.string2Bytes(
+              kpTokenIssuer.getKeyProviderUri().toString()));
+      if (tokens != null && kpTokens != null) {
+        Token<?>[] all = new Token<?>[tokens.length + kpTokens.length];
+        System.arraycopy(tokens, 0, all, 0, tokens.length);
+        System.arraycopy(kpTokens, 0, all, tokens.length, kpTokens.length);
+        tokens = all;
+      } else {
+        tokens = (tokens != null) ? tokens : kpTokens;
+      }
+    }
+    return tokens;
+  }
+
+  /**
+   * Obtain the crypto protocol version from the provided FileEncryptionInfo,
+   * checking to see if this version is supported by.
+   *
+   * @param feInfo FileEncryptionInfo
+   * @return CryptoProtocolVersion from the feInfo
+   * @throws IOException if the protocol version is unsupported.
+   */
+  public static CryptoProtocolVersion getCryptoProtocolVersion(
+      FileEncryptionInfo feInfo) throws IOException {
+    final CryptoProtocolVersion version = feInfo.getCryptoProtocolVersion();
+    if (!CryptoProtocolVersion.supports(version)) {
+      throw new IOException("Client does not support specified " +
+          "CryptoProtocolVersion " + version.getDescription() + " version " +
+          "number" + version.getVersion());
+    }
+    return version;
+  }
+
+  /**
+   * Obtain a CryptoCodec based on the CipherSuite set in a FileEncryptionInfo
+   * and the available CryptoCodecs configured in the Configuration.
+   *
+   * @param conf   Configuration
+   * @param feInfo FileEncryptionInfo
+   * @return CryptoCodec
+   * @throws IOException if no suitable CryptoCodec for the CipherSuite is
+   *                     available.
+   */
+  public static CryptoCodec getCryptoCodec(Configuration conf,
+      FileEncryptionInfo feInfo) throws IOException {
+    final CipherSuite suite = feInfo.getCipherSuite();
+    if (suite.equals(CipherSuite.UNKNOWN)) {
+      throw new IOException("NameNode specified unknown CipherSuite with ID "
+          + suite.getUnknownValue() + ", cannot instantiate CryptoCodec.");
+    }
+    final CryptoCodec codec = CryptoCodec.getInstance(conf, suite);
+    if (codec == null) {
+      throw new UnknownCipherSuiteException(
+          "No configuration found for the cipher suite "
+              + suite.getConfigSuffix() + " prefixed with "
+              + HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
+              + ". Please see the example configuration "
+              + "hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE "
+              + "at core-default.xml for details.");
+    }
+    return codec;
+  }
+
+  /**
+   * The key provider uri is searched in the following order.
+   * 1. If there is a mapping in Credential's secrets map for namenode uri.
+   * 2. From namenode getServerDefaults call.
+   * 3. Finally fallback to local conf.
+   * @return keyProviderUri if found from either of above 3 cases,
+   * null otherwise
+   * @throws IOException
+   */
+  public static URI getKeyProviderUri(UserGroupInformation ugi,
+      URI namenodeUri, String keyProviderUriStr, Configuration conf)
+          throws IOException {
+    URI keyProviderUri = null;
+    // Lookup the secret in credentials object for namenodeuri.
+    Credentials credentials = ugi.getCredentials();
+    byte[] keyProviderUriBytes =
+        credentials.getSecretKey(getKeyProviderMapKey(namenodeUri));
+    if(keyProviderUriBytes != null) {
+      keyProviderUri =
+          URI.create(DFSUtilClient.bytes2String(keyProviderUriBytes));
+      return keyProviderUri;
+    }
+
+    if (keyProviderUriStr != null) {
+      if (!keyProviderUriStr.isEmpty()) {
+        keyProviderUri = URI.create(keyProviderUriStr);
+      }
+      return keyProviderUri;
+    }
+
+    // Last thing is to trust its own conf to be backwards compatible.
+    String keyProviderUriFromConf = conf.getTrimmed(
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH);
+    if (keyProviderUriFromConf != null && !keyProviderUriFromConf.isEmpty()) {
+      keyProviderUri = URI.create(keyProviderUriFromConf);
+    }
+    return keyProviderUri;
+  }
+
+  /**
+   * Returns a key to map namenode uri to key provider uri.
+   * Tasks will lookup this key to find key Provider.
+   */
+  public static Text getKeyProviderMapKey(URI namenodeUri) {
+    return new Text(DFS_KMS_PREFIX + namenodeUri.getScheme()
+        +"://" + namenodeUri.getAuthority());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/404eab4d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index c1aef49..2ab7a83 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -56,6 +56,8 @@ import com.fasterxml.jackson.databind.ObjectReader;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.input.BoundedInputStream;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.KeyProviderTokenIssuer;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.ContentSummary;
@@ -84,6 +86,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HAUtilClient;
+import org.apache.hadoop.hdfs.HdfsKMSUtil;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
@@ -100,6 +103,7 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
@@ -107,6 +111,7 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenSelector;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
+import org.apache.hadoop.util.KMSUtil;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
@@ -120,7 +125,7 @@ import com.google.common.collect.Lists;
 /** A FileSystem for HDFS over the web. */
 public class WebHdfsFileSystem extends FileSystem
     implements DelegationTokenRenewer.Renewable,
-    TokenAspect.TokenManagementDelegator {
+    TokenAspect.TokenManagementDelegator, KeyProviderTokenIssuer {
   public static final Logger LOG = LoggerFactory
       .getLogger(WebHdfsFileSystem.class);
   /** WebHdfs version. */
@@ -1633,6 +1638,13 @@ public class WebHdfsFileSystem extends FileSystem
   }
 
   @Override
+  public Token<?>[] addDelegationTokens(String renewer,
+      Credentials credentials) throws IOException {
+    Token<?>[] tokens = super.addDelegationTokens(renewer, credentials);
+    return HdfsKMSUtil.addDelegationTokensForKeyProvider(this, renewer,
+        credentials, getUri(), tokens);
+  }
+
   public BlockLocation[] getFileBlockLocations(final FileStatus status,
       final long offset, final long length) throws IOException {
     if (status == null) {
@@ -1822,6 +1834,29 @@ public class WebHdfsFileSystem extends FileSystem
     this.retryPolicy = rp;
   }
 
+
+  @Override
+  public URI getKeyProviderUri() throws IOException {
+    String keyProviderUri = null;
+    try {
+      keyProviderUri = getServerDefaults().getKeyProviderUri();
+    } catch (UnsupportedOperationException e) {
+      // This means server doesn't supports GETSERVERDEFAULTS call.
+      // Do nothing, let keyProviderUri = null.
+    }
+    return HdfsKMSUtil.getKeyProviderUri(ugi, getUri(), keyProviderUri,
+        getConf());
+  }
+
+  @Override
+  public KeyProvider getKeyProvider() throws IOException {
+    URI keyProviderUri = getKeyProviderUri();
+    if (keyProviderUri == null) {
+      return null;
+    }
+    return KMSUtil.createKeyProviderFromUri(getConf(), keyProviderUri);
+  }
+
   /**
    * This class is used for opening, reading, and seeking files while using the
    * WebHdfsFileSystem. This class will invoke the retry policy when performing

http://git-wip-us.apache.org/repos/asf/hadoop/blob/404eab4d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 3f6c3d7..4c94e38 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -1655,7 +1655,7 @@ public class DFSUtil {
    */
   public static KeyProviderCryptoExtension createKeyProviderCryptoExtension(
       final Configuration conf) throws IOException {
-    KeyProvider keyProvider = DFSUtilClient.createKeyProvider(conf);
+    KeyProvider keyProvider = HdfsKMSUtil.createKeyProvider(conf);
     if (keyProvider == null) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/404eab4d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
index 3ed6b63..4497e23 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
@@ -1705,7 +1705,8 @@ public class TestEncryptionZones {
     Credentials credentials = new Credentials();
     // Key provider uri should be in the secret map of credentials object with
     // namenode uri as key
-    Text lookUpKey = client.getKeyProviderMapKey();
+    Text lookUpKey = HdfsKMSUtil.getKeyProviderMapKey(
+        cluster.getFileSystem().getUri());
     credentials.addSecretKey(lookUpKey,
         DFSUtilClient.string2Bytes(dummyKeyProvider));
     client.ugi.addCredentials(credentials);
@@ -1856,7 +1857,8 @@ public class TestEncryptionZones {
         CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH);
     DFSClient client = cluster.getFileSystem().getClient();
     Credentials credentials = new Credentials();
-    Text lookUpKey = client.getKeyProviderMapKey();
+    Text lookUpKey = HdfsKMSUtil.
+        getKeyProviderMapKey(cluster.getFileSystem().getUri());
     credentials.addSecretKey(lookUpKey,
         DFSUtilClient.string2Bytes(getKeyProviderURI()));
     client.ugi.addCredentials(credentials);
@@ -1920,4 +1922,38 @@ public class TestEncryptionZones {
         dfsAdmin.listEncryptionZones().hasNext());
   }
 
+  /**
+  * This test returns mocked kms token when
+  * {@link WebHdfsFileSystem#addDelegationTokens(String, Credentials)} method
+  * is called.
+  * @throws Exception
+  */
+  @Test
+  public void addMockKmsToken() throws Exception {
+    UserGroupInformation.createRemoteUser("JobTracker");
+    WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
+        WebHdfsConstants.WEBHDFS_SCHEME);
+    KeyProvider keyProvider = Mockito.mock(KeyProvider.class, withSettings()
+        .extraInterfaces(DelegationTokenExtension.class,
+         CryptoExtension.class));
+    Mockito.when(keyProvider.getConf()).thenReturn(conf);
+    byte[] testIdentifier = "Test identifier for delegation token".getBytes();
+
+    Token<?> testToken = new Token(testIdentifier, new byte[0],
+        new Text("kms-dt"), new Text());
+    Mockito.when(((DelegationTokenExtension) keyProvider)
+        .addDelegationTokens(anyString(), (Credentials) any()))
+        .thenReturn(new Token<?>[] {testToken});
+
+    WebHdfsFileSystem webfsSpy = Mockito.spy(webfs);
+    Mockito.doReturn(keyProvider).when(webfsSpy).getKeyProvider();
+
+    Credentials creds = new Credentials();
+    final Token<?>[] tokens =
+        webfsSpy.addDelegationTokens("JobTracker", creds);
+
+    Assert.assertEquals(2, tokens.length);
+    Assert.assertEquals(tokens[1], testToken);
+    Assert.assertEquals(1, creds.numberOfTokens());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/404eab4d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithKMS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithKMS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithKMS.java
index 6f53362..16ab073 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithKMS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithKMS.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertTrue;
 
 import com.google.common.base.Supplier;
 import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
+import org.apache.hadoop.crypto.key.kms.KMSDelegationToken;
 import org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider;
 import org.apache.hadoop.crypto.key.kms.server.MiniKMS;
 import org.apache.hadoop.security.Credentials;
@@ -28,6 +29,9 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.web.WebHdfsConstants;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Assert;
@@ -134,4 +138,23 @@ public class TestEncryptionZonesWithKMS extends TestEncryptionZones {
       }
     }, 1000, 60000);
   }
+
+  /**
+   * This method fetches the kms delegation token
+   * for {@link WebHdfsFileSystem}.
+   * @throws Exception
+   */
+  @Test
+  public void addDelegationTokenFromWebhdfsFileSystem() throws Exception {
+    UserGroupInformation.createRemoteUser("JobTracker");
+    WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystem(
+        conf, WebHdfsConstants.WEBHDFS_SCHEME);
+    Credentials creds = new Credentials();
+    final Token<?>[] tokens = webfs.addDelegationTokens("JobTracker", creds);
+
+    Assert.assertEquals(2, tokens.length);
+    Assert.assertEquals(KMSDelegationToken.TOKEN_KIND_STR,
+        tokens[1].getKind().toString());
+    Assert.assertEquals(2, creds.numberOfTokens());
+  }
 }


---------------------------------------------------------------------
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: HDFS-12594. snapshotDiff fails if the report exceeds the RPC response limit. Contributed by Shashikant Banerjee

Posted by ae...@apache.org.
HDFS-12594. snapshotDiff fails if the report exceeds the RPC response limit. Contributed by Shashikant Banerjee


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

Branch: refs/heads/HDFS-7240
Commit: b1c7654ee40b372ed777525a42981c7cf55b5c72
Parents: 5cfaee2
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Thu Nov 30 12:18:29 2017 -0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Thu Nov 30 12:18:29 2017 -0800

----------------------------------------------------------------------
 .../dev-support/findbugsExcludeFile.xml         |   2 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  14 +-
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  57 +++-
 .../hadoop/hdfs/DistributedFileSystem.java      |  38 ++-
 .../impl/SnapshotDiffReportGenerator.java       | 262 +++++++++++++++++++
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  29 ++
 .../protocol/SnapshotDiffReportListing.java     | 160 +++++++++++
 .../ClientNamenodeProtocolTranslatorPB.java     |  24 ++
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  | 127 +++++++++
 .../src/main/proto/ClientNamenodeProtocol.proto |  12 +
 .../src/main/proto/hdfs.proto                   |  26 ++
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   5 +
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |  42 +--
 ...tNamenodeProtocolServerSideTranslatorPB.java |  22 ++
 .../federation/router/RouterRpcServer.java      |   9 +
 .../hdfs/server/namenode/FSDirSnapshotOp.java   |  24 ++
 .../hdfs/server/namenode/FSNamesystem.java      |  77 +++++-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  13 +
 .../snapshot/DirectorySnapshottableFeature.java | 136 +++++++++-
 .../snapshot/SnapshotDiffListingInfo.java       | 207 +++++++++++++++
 .../namenode/snapshot/SnapshotManager.java      |  28 ++
 .../src/main/resources/hdfs-default.xml         |  11 +
 .../snapshot/TestSnapshotDiffReport.java        | 116 ++++++++
 23 files changed, 1384 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
index 22ef722..8e2bc94 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
@@ -19,6 +19,8 @@
       <Class name="org.apache.hadoop.hdfs.DFSPacket"/>
       <Class name="org.apache.hadoop.hdfs.protocol.LocatedStripedBlock"/>
       <Class name="org.apache.hadoop.hdfs.util.StripedBlockUtil$ChunkByteArray"/>
+      <Class name="org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing$DiffReportListingEntry"/>
+      <Class name="org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing"/>
     </Or>
     <Bug pattern="EI_EXPOSE_REP,EI_EXPOSE_REP2" />
   </Match>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 25e0f6c..3df36d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -139,10 +139,10 @@ import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
 import org.apache.hadoop.hdfs.protocol.ReencryptionStatusIterator;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
@@ -2140,14 +2140,16 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   /**
    * Get the difference between two snapshots, or between a snapshot and the
    * current tree of a directory.
-   * @see ClientProtocol#getSnapshotDiffReport(String, String, String)
+   * @see ClientProtocol#getSnapshotDiffReportListing
    */
-  public SnapshotDiffReport getSnapshotDiffReport(String snapshotDir,
-      String fromSnapshot, String toSnapshot) throws IOException {
+  public SnapshotDiffReportListing getSnapshotDiffReportListing(
+      String snapshotDir, String fromSnapshot, String toSnapshot,
+      byte[] startPath, int index) throws IOException {
     checkOpen();
     try (TraceScope ignored = tracer.newScope("getSnapshotDiffReport")) {
-      return namenode.getSnapshotDiffReport(snapshotDir,
-          fromSnapshot, toSnapshot);
+      return namenode
+          .getSnapshotDiffReportListing(snapshotDir, fromSnapshot, toSnapshot,
+              startPath, index);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
index 2a8bf0d..f6b28e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
@@ -89,6 +89,7 @@ import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.Arrays;
 
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_CLIENT_TCPNODELAY_DEFAULT;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_CLIENT_TCPNODELAY_KEY;
@@ -124,6 +125,56 @@ public class DFSUtilClient {
     return bytes2String(bytes, 0, bytes.length);
   }
 
+  /**
+   * Converts a byte array to array of arrays of bytes
+   * on byte separator.
+   */
+  public static byte[][] bytes2byteArray(byte[] bytes) {
+    return bytes2byteArray(bytes, bytes.length, (byte)Path.SEPARATOR_CHAR);
+  }
+  /**
+   * Splits first len bytes in bytes to array of arrays of bytes
+   * on byte separator.
+   * @param bytes the byte array to split
+   * @param len the number of bytes to split
+   * @param separator the delimiting byte
+   */
+  public static byte[][] bytes2byteArray(byte[] bytes, int len,
+      byte separator) {
+    Preconditions.checkPositionIndex(len, bytes.length);
+    if (len == 0) {
+      return new byte[][]{null};
+    }
+    // Count the splits. Omit multiple separators and the last one by
+    // peeking at prior byte.
+    int splits = 0;
+    for (int i = 1; i < len; i++) {
+      if (bytes[i-1] == separator && bytes[i] != separator) {
+        splits++;
+      }
+    }
+    if (splits == 0 && bytes[0] == separator) {
+      return new byte[][]{null};
+    }
+    splits++;
+    byte[][] result = new byte[splits][];
+    int nextIndex = 0;
+    // Build the splits.
+    for (int i = 0; i < splits; i++) {
+      int startIndex = nextIndex;
+      // find next separator in the bytes.
+      while (nextIndex < len && bytes[nextIndex] != separator) {
+        nextIndex++;
+      }
+      result[i] = (nextIndex > 0)
+          ? Arrays.copyOfRange(bytes, startIndex, nextIndex)
+          : DFSUtilClient.EMPTY_BYTES; // reuse empty bytes for root.
+      do { // skip over separators.
+        nextIndex++;
+      } while (nextIndex < len && bytes[nextIndex] == separator);
+    }
+    return result;
+  }
   /** Return used as percentage of capacity */
   public static float getPercentUsed(long used, long capacity) {
     return capacity <= 0 ? 100 : (used * 100.0f)/capacity;
@@ -277,11 +328,9 @@ public class DFSUtilClient {
    * Given a list of path components returns a byte array
    */
   public static byte[] byteArray2bytes(byte[][] pathComponents) {
-    if (pathComponents.length == 0) {
+    if (pathComponents.length == 0 ||  (pathComponents.length == 1
+        && (pathComponents[0] == null || pathComponents[0].length == 0))) {
       return EMPTY_BYTES;
-    } else if (pathComponents.length == 1
-        && (pathComponents[0] == null || pathComponents[0].length == 0)) {
-      return new byte[]{(byte) Path.SEPARATOR_CHAR};
     }
     int length = 0;
     for (int i = 0; i < pathComponents.length; i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 9db12e1..c010c8a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import org.apache.commons.collections.list.TreeList;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -90,12 +91,16 @@ import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing.DiffReportListingEntry;
+import org.apache.hadoop.hdfs.client.impl.SnapshotDiffReportGenerator;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.ChunkedArrayList;
 import org.apache.hadoop.util.Progressable;
 
 import javax.annotation.Nonnull;
@@ -1971,19 +1976,46 @@ public class DistributedFileSystem extends FileSystem {
     }.resolve(this, absF);
   }
 
+  private SnapshotDiffReport getSnapshotDiffReportInternal(
+      final String snapshotDir, final String fromSnapshot,
+      final String toSnapshot) throws IOException {
+    byte[] startPath = DFSUtilClient.EMPTY_BYTES;
+    int index = -1;
+    SnapshotDiffReportGenerator snapshotDiffReport;
+    List<DiffReportListingEntry> modifiedList = new TreeList();
+    List<DiffReportListingEntry> createdList = new ChunkedArrayList<>();
+    List<DiffReportListingEntry> deletedList = new ChunkedArrayList<>();
+    SnapshotDiffReportListing report;
+    do {
+      report = dfs.getSnapshotDiffReportListing(snapshotDir, fromSnapshot,
+          toSnapshot, startPath, index);
+      startPath = report.getLastPath();
+      index = report.getLastIndex();
+      modifiedList.addAll(report.getModifyList());
+      createdList.addAll(report.getCreateList());
+      deletedList.addAll(report.getDeleteList());
+    } while (!(Arrays.equals(startPath, DFSUtilClient.EMPTY_BYTES)
+        && index == -1));
+    snapshotDiffReport =
+        new SnapshotDiffReportGenerator(snapshotDir, fromSnapshot, toSnapshot,
+            report.getIsFromEarlier(), modifiedList, createdList, deletedList);
+    return snapshotDiffReport.generateReport();
+  }
+
   /**
    * Get the difference between two snapshots, or between a snapshot and the
    * current tree of a directory.
    *
-   * @see DFSClient#getSnapshotDiffReport(String, String, String)
+   * @see DFSClient#getSnapshotDiffReportListing
    */
   public SnapshotDiffReport getSnapshotDiffReport(final Path snapshotDir,
       final String fromSnapshot, final String toSnapshot) throws IOException {
     Path absF = fixRelativePart(snapshotDir);
     return new FileSystemLinkResolver<SnapshotDiffReport>() {
       @Override
-      public SnapshotDiffReport doCall(final Path p) throws IOException {
-        return dfs.getSnapshotDiffReport(getPathName(p), fromSnapshot,
+      public SnapshotDiffReport doCall(final Path p)
+          throws IOException {
+        return getSnapshotDiffReportInternal(getPathName(p), fromSnapshot,
             toSnapshot);
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/SnapshotDiffReportGenerator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/SnapshotDiffReportGenerator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/SnapshotDiffReportGenerator.java
new file mode 100644
index 0000000..4dbe988
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/SnapshotDiffReportGenerator.java
@@ -0,0 +1,262 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.client.impl;
+
+import java.util.*;
+
+import com.google.common.primitives.SignedBytes;
+
+import org.apache.hadoop.util.ChunkedArrayList;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing.DiffReportListingEntry;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
+/**
+ * This class represents to end users the difference between two snapshots of
+ * the same directory, or the difference between a snapshot of the directory and
+ * its current state. Instead of capturing all the details of the diff, this
+ * class only lists where the changes happened and their types.
+ */
+public class SnapshotDiffReportGenerator {
+  /**
+   * Compare two inodes based on their full names.
+   */
+  public static final Comparator<DiffReportListingEntry> INODE_COMPARATOR =
+      new Comparator<DiffReportListingEntry>() {
+        @Override
+        public int compare(DiffReportListingEntry left,
+            DiffReportListingEntry right) {
+          final Comparator<byte[]> cmp =
+              SignedBytes.lexicographicalComparator();
+          //source path can never be null
+          final byte[][] l = left.getSourcePath();
+          final byte[][] r = right.getSourcePath();
+          if (l.length == 1 && l[0] == null) {
+            return -1;
+          } else if (r.length == 1 && r[0] == null) {
+            return 1;
+          } else {
+            for (int i = 0; i < l.length && i < r.length; i++) {
+              final int diff = cmp.compare(l[i], r[i]);
+              if (diff != 0) {
+                return diff;
+              }
+            }
+            return l.length == r.length ? 0 : l.length > r.length ? 1 : -1;
+          }
+        }
+      };
+
+  static class RenameEntry {
+    private byte[][] sourcePath;
+    private byte[][] targetPath;
+
+    void setSource(byte[][] srcPath) {
+      this.sourcePath = srcPath;
+    }
+
+    void setTarget(byte[][] target) {
+      this.targetPath = target;
+    }
+
+    boolean isRename() {
+      return sourcePath != null && targetPath != null;
+    }
+
+    byte[][] getSourcePath() {
+      return sourcePath;
+    }
+
+    byte[][] getTargetPath() {
+      return targetPath;
+    }
+  }
+
+  /*
+   * A class represnting the diff in a directory between two given snapshots
+   * in two lists: createdList and deleted list.
+   */
+  static class ChildrenDiff {
+    private final List<DiffReportListingEntry> createdList;
+    private final List<DiffReportListingEntry> deletedList;
+
+    ChildrenDiff(List<DiffReportListingEntry> createdList,
+        List<DiffReportListingEntry> deletedList) {
+      this.createdList = createdList != null ? createdList :
+          Collections.emptyList();
+      this.deletedList = deletedList != null ? deletedList :
+          Collections.emptyList();
+    }
+
+    public List<DiffReportListingEntry> getCreatedList() {
+      return createdList;
+    }
+
+    public List<DiffReportListingEntry> getDeletedList() {
+      return deletedList;
+    }
+  }
+
+  /**
+   * snapshot root full path.
+   */
+  private final String snapshotRoot;
+
+  /**
+   * start point of the diff.
+   */
+  private final String fromSnapshot;
+
+  /**
+   * end point of the diff.
+   */
+  private final String toSnapshot;
+
+  /**
+   * Flag to indicate the diff is calculated from older to newer snapshot
+   * or not.
+   */
+  private final boolean isFromEarlier;
+
+  /**
+   * A map capturing the detailed difference about file creation/deletion.
+   * Each key indicates a directory inode whose children have been changed
+   * between the two snapshots, while its associated value is a
+   * {@link ChildrenDiff} storing the changes (creation/deletion) happened to
+   * the children (files).
+   */
+  private final Map<Long, ChildrenDiff> dirDiffMap =
+      new HashMap<>();
+
+  private final Map<Long, RenameEntry> renameMap =
+      new HashMap<>();
+
+  private List<DiffReportListingEntry> mlist = null;
+  private List<DiffReportListingEntry> clist = null;
+  private List<DiffReportListingEntry> dlist = null;
+
+  public SnapshotDiffReportGenerator(String snapshotRoot, String fromSnapshot,
+      String toSnapshot, boolean isFromEarlier,
+      List<DiffReportListingEntry> mlist, List<DiffReportListingEntry> clist,
+      List<DiffReportListingEntry> dlist) {
+    this.snapshotRoot = snapshotRoot;
+    this.fromSnapshot = fromSnapshot;
+    this.toSnapshot = toSnapshot;
+    this.isFromEarlier = isFromEarlier;
+    this.mlist =
+        mlist != null ? mlist : Collections.emptyList();
+    this.clist =
+        clist != null ? clist : Collections.emptyList();
+    this.dlist =
+        dlist != null ? dlist : Collections.emptyList();
+  }
+
+  private RenameEntry getEntry(long inodeId) {
+    RenameEntry entry = renameMap.get(inodeId);
+    if (entry == null) {
+      entry = new RenameEntry();
+      renameMap.put(inodeId, entry);
+    }
+    return entry;
+  }
+
+  public void generateReportList() {
+    mlist.sort(INODE_COMPARATOR);
+    for (DiffReportListingEntry created : clist) {
+      ChildrenDiff entry = dirDiffMap.get(created.getDirId());
+      if (entry == null) {
+        List<DiffReportListingEntry> createdList = new ChunkedArrayList<>();
+        createdList.add(created);
+        ChildrenDiff list = new ChildrenDiff(createdList, null);
+        dirDiffMap.put(created.getDirId(), list);
+      } else {
+        dirDiffMap.get(created.getDirId()).getCreatedList().add(created);
+      }
+      if (created.isReference()) {
+        RenameEntry renameEntry = getEntry(created.getFileId());
+        if (renameEntry.getTargetPath() != null) {
+          renameEntry.setTarget(created.getSourcePath());
+        }
+      }
+    }
+    for (DiffReportListingEntry deleted : dlist) {
+      ChildrenDiff entry = dirDiffMap.get(deleted.getDirId());
+      if (entry == null || (entry.getDeletedList().isEmpty())) {
+        ChildrenDiff list;
+        List<DiffReportListingEntry> deletedList = new ChunkedArrayList<>();
+        deletedList.add(deleted);
+        if (entry == null) {
+          list = new ChildrenDiff(null, deletedList);
+        } else {
+          list = new ChildrenDiff(entry.getCreatedList(), deletedList);
+        }
+        dirDiffMap.put(deleted.getDirId(), list);
+      } else {
+        entry.getDeletedList().add(deleted);
+      }
+      if (deleted.isReference()) {
+        RenameEntry renameEntry = getEntry(deleted.getFileId());
+        renameEntry.setTarget(deleted.getTargetPath());
+        renameEntry.setSource(deleted.getSourcePath());
+      }
+    }
+  }
+
+  public SnapshotDiffReport generateReport() {
+    List<DiffReportEntry> diffReportList = new ChunkedArrayList<>();
+    generateReportList();
+    for (DiffReportListingEntry modified : mlist) {
+      diffReportList.add(
+          new DiffReportEntry(DiffType.MODIFY, modified.getSourcePath(), null));
+      if (modified.isReference()
+          && dirDiffMap.get(modified.getDirId()) != null) {
+        List<DiffReportEntry> subList = generateReport(modified);
+        diffReportList.addAll(subList);
+      }
+    }
+    return new SnapshotDiffReport(snapshotRoot, fromSnapshot, toSnapshot,
+        diffReportList);
+  }
+
+  private List<DiffReportEntry> generateReport(
+      DiffReportListingEntry modified) {
+    List<DiffReportEntry> diffReportList = new ChunkedArrayList<>();
+    ChildrenDiff list = dirDiffMap.get(modified.getDirId());
+    for (DiffReportListingEntry created : list.getCreatedList()) {
+      RenameEntry entry = renameMap.get(created.getFileId());
+      if (entry == null || !entry.isRename()) {
+        diffReportList.add(new DiffReportEntry(
+            isFromEarlier ? DiffType.CREATE : DiffType.DELETE,
+            created.getSourcePath()));
+      }
+    }
+    for (DiffReportListingEntry deleted : list.getDeletedList()) {
+      RenameEntry entry = renameMap.get(deleted.getFileId());
+      if (entry != null && entry.isRename()) {
+        diffReportList.add(new DiffReportEntry(DiffType.RENAME,
+            isFromEarlier ? entry.getSourcePath() : entry.getTargetPath(),
+            isFromEarlier ? entry.getTargetPath() : entry.getSourcePath()));
+      } else {
+        diffReportList.add(new DiffReportEntry(
+            isFromEarlier ? DiffType.DELETE : DiffType.CREATE,
+            deleted.getSourcePath()));
+      }
+    }
+    return diffReportList;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index f61ec75..eb2e11c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1289,6 +1289,35 @@ public interface ClientProtocol {
       String fromSnapshot, String toSnapshot) throws IOException;
 
   /**
+   * Get the difference between two snapshots, or between a snapshot and the
+   * current tree of a directory.
+   *
+   * @param snapshotRoot
+   *          full path of the directory where snapshots are taken
+   * @param fromSnapshot
+   *          snapshot name of the from point. Null indicates the current
+   *          tree
+   * @param toSnapshot
+   *          snapshot name of the to point. Null indicates the current
+   *          tree.
+   * @param startPath
+   *          path relative to the snapshottable root directory from where the
+   *          snapshotdiff computation needs to start across multiple rpc calls
+   * @param index
+   *           index in the created or deleted list of the directory at which
+   *           the snapshotdiff computation stopped during the last rpc call
+   *           as the no of entries exceeded the snapshotdiffentry limit. -1
+   *           indicates, the snapshotdiff compuatation needs to start right
+   *           from the startPath provided.
+   * @return The difference report represented as a {@link SnapshotDiffReport}.
+   * @throws IOException on error
+   */
+  @Idempotent
+  SnapshotDiffReportListing getSnapshotDiffReportListing(String snapshotRoot,
+      String fromSnapshot, String toSnapshot, byte[] startPath, int index)
+      throws IOException;
+
+  /**
    * Add a CacheDirective to the CacheManager.
    *
    * @param directive A CacheDirectiveInfo to be added

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReportListing.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReportListing.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReportListing.java
new file mode 100644
index 0000000..a0e35f6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReportListing.java
@@ -0,0 +1,160 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.curator.shaded.com.google.common.base.Preconditions;
+
+import org.apache.hadoop.hdfs.DFSUtilClient;
+
+/**
+ * This class represents to  the difference between two snapshots of
+ * the same directory, or the difference between a snapshot of the directory and
+ * its current state. This Class serves the purpose of collecting diff entries
+ * in 3 lists : created, deleted and modified list combined size of which is set
+ * by dfs.snapshotdiff-report.limit over one rpc call to the namenode.
+ */
+public class SnapshotDiffReportListing {
+  /**
+   * Representing the full path and diff type of a file/directory where changes
+   * have happened.
+   */
+  public static class DiffReportListingEntry {
+    /**
+     * The type of the difference.
+     */
+    private final long fileId;
+    private final long dirId;
+    private final boolean isReference;
+    /**
+     * The relative path (related to the snapshot root) of 1) the file/directory
+     * where changes have happened, or 2) the source file/dir of a rename op.
+     * or 3) target file/dir for a rename op.
+     */
+    private final byte[][] sourcePath;
+    private final byte[][] targetPath;
+
+    public DiffReportListingEntry(long dirId, long fileId, byte[][] sourcePath,
+        boolean isReference, byte[][] targetPath) {
+      Preconditions.checkNotNull(sourcePath);
+      this.dirId = dirId;
+      this.fileId = fileId;
+      this.sourcePath = sourcePath;
+      this.isReference = isReference;
+      this.targetPath = targetPath;
+    }
+
+    public DiffReportListingEntry(long dirId, long fileId, byte[] sourcePath,
+        boolean isReference, byte[] targetpath) {
+      Preconditions.checkNotNull(sourcePath);
+      this.dirId = dirId;
+      this.fileId = fileId;
+      this.sourcePath = DFSUtilClient.bytes2byteArray(sourcePath);
+      this.isReference = isReference;
+      this.targetPath =
+          targetpath == null ? null : DFSUtilClient.bytes2byteArray(targetpath);
+    }
+
+    public long getDirId() {
+      return dirId;
+    }
+
+    public long getFileId() {
+      return fileId;
+    }
+
+    public byte[][] getSourcePath() {
+      return sourcePath;
+    }
+
+    public byte[][] getTargetPath() {
+      return targetPath;
+    }
+
+    public boolean isReference() {
+      return isReference;
+    }
+  }
+
+  /** store the starting path to process across RPC's for snapshot diff. */
+  private final byte[] lastPath;
+
+  private final int lastIndex;
+
+  private final boolean isFromEarlier;
+
+  /** list of diff. */
+  private final List<DiffReportListingEntry> modifyList;
+
+  private final List<DiffReportListingEntry> createList;
+
+  private final List<DiffReportListingEntry> deleteList;
+
+  public SnapshotDiffReportListing() {
+    this.modifyList = Collections.emptyList();
+    this.createList = Collections.emptyList();
+    this.deleteList = Collections.emptyList();
+    this.lastPath = DFSUtilClient.string2Bytes("");
+    this.lastIndex = -1;
+    this.isFromEarlier = false;
+  }
+
+  public SnapshotDiffReportListing(byte[] startPath,
+      List<DiffReportListingEntry> modifiedEntryList,
+      List<DiffReportListingEntry> createdEntryList,
+      List<DiffReportListingEntry> deletedEntryList, int index,
+      boolean isFromEarlier) {
+    this.modifyList = modifiedEntryList;
+    this.createList = createdEntryList;
+    this.deleteList = deletedEntryList;
+    this.lastPath =
+        startPath != null ? startPath : DFSUtilClient.string2Bytes("");
+    this.lastIndex = index;
+    this.isFromEarlier = isFromEarlier;
+  }
+
+  public List<DiffReportListingEntry> getModifyList() {
+    return modifyList;
+  }
+
+  public List<DiffReportListingEntry> getCreateList() {
+    return createList;
+  }
+
+  public List<DiffReportListingEntry> getDeleteList() {
+    return deleteList;
+  }
+
+  /**
+   * @return {@link #lastPath}
+   */
+  public byte[] getLastPath() {
+    return lastPath;
+  }
+
+  public int getLastIndex() {
+    return lastIndex;
+  }
+
+  public boolean getIsFromEarlier() {
+    return isFromEarlier;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index aef7c1e..38dc44b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -79,6 +79,7 @@ import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto;
@@ -133,6 +134,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetQuo
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesRequestProto;
@@ -1206,6 +1209,27 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
+  public SnapshotDiffReportListing getSnapshotDiffReportListing(
+      String snapshotRoot, String fromSnapshot, String toSnapshot,
+      byte[] startPath, int index) throws IOException {
+    GetSnapshotDiffReportListingRequestProto req =
+        GetSnapshotDiffReportListingRequestProto.newBuilder()
+            .setSnapshotRoot(snapshotRoot).setFromSnapshot(fromSnapshot)
+            .setToSnapshot(toSnapshot).setCursor(
+            HdfsProtos.SnapshotDiffReportCursorProto.newBuilder()
+                .setStartPath(PBHelperClient.getByteString(startPath))
+                .setIndex(index).build()).build();
+    try {
+      GetSnapshotDiffReportListingResponseProto result =
+          rpcProxy.getSnapshotDiffReportListing(null, req);
+
+      return PBHelperClient.convert(result.getDiffReport());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
   public long addCacheDirective(CacheDirectiveInfo directive,
       EnumSet<CacheFlag> flags) throws IOException {
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index d3b7f6d..fbc6dbf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -99,6 +99,8 @@ import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing.DiffReportListingEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
@@ -169,6 +171,8 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.QuotaUsageProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReencryptionInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RollingUpgradeStatusProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportListingEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportListingProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto;
@@ -1489,6 +1493,61 @@ public class PBHelperClient {
         .toByteArray() : null);
   }
 
+  public static SnapshotDiffReportListing convert(
+      SnapshotDiffReportListingProto reportProto) {
+    if (reportProto == null) {
+      return null;
+    }
+    List<SnapshotDiffReportListingEntryProto> modifyList =
+        reportProto.getModifiedEntriesList();
+    List<DiffReportListingEntry> modifiedEntries = new ChunkedArrayList<>();
+    for (SnapshotDiffReportListingEntryProto entryProto : modifyList) {
+      DiffReportListingEntry entry = convert(entryProto);
+      if (entry != null) {
+        modifiedEntries.add(entry);
+      }
+    }
+    List<SnapshotDiffReportListingEntryProto> createList =
+        reportProto.getCreatedEntriesList();
+    List<DiffReportListingEntry> createdEntries = new ChunkedArrayList<>();
+    for (SnapshotDiffReportListingEntryProto entryProto : createList) {
+      DiffReportListingEntry entry = convert(entryProto);
+      if (entry != null) {
+        createdEntries.add(entry);
+      }
+    }
+    List<SnapshotDiffReportListingEntryProto> deletedList =
+        reportProto.getDeletedEntriesList();
+    List<DiffReportListingEntry> deletedEntries = new ChunkedArrayList<>();
+    for (SnapshotDiffReportListingEntryProto entryProto : deletedList) {
+      DiffReportListingEntry entry = convert(entryProto);
+      if (entry != null) {
+        deletedEntries.add(entry);
+      }
+    }
+    byte[] startPath = reportProto.getCursor().getStartPath().toByteArray();
+    boolean isFromEarlier = reportProto.getIsFromEarlier();
+
+    int index = reportProto.getCursor().getIndex();
+    return new SnapshotDiffReportListing(startPath, modifiedEntries,
+        createdEntries, deletedEntries, index, isFromEarlier);
+  }
+
+  public static DiffReportListingEntry convert(
+      SnapshotDiffReportListingEntryProto entry) {
+    if (entry == null) {
+      return null;
+    }
+    long dirId = entry.getDirId();
+    long fileId = entry.getFileId();
+    boolean isReference = entry.getIsReference();
+    byte[] sourceName = entry.getFullpath().toByteArray();
+    byte[] targetName =
+        entry.hasTargetPath() ? entry.getTargetPath().toByteArray() : null;
+    return new DiffReportListingEntry(dirId, fileId, sourceName, isReference,
+        targetName);
+  }
+
   public static SnapshottableDirectoryStatus[] convert(
       SnapshottableDirectoryListingProto sdlp) {
     if (sdlp == null)
@@ -2508,6 +2567,74 @@ public class PBHelperClient {
     return builder.build();
   }
 
+  public static SnapshotDiffReportListingEntryProto convert(
+      DiffReportListingEntry entry) {
+    if (entry == null) {
+      return null;
+    }
+    ByteString sourcePath = getByteString(
+        entry.getSourcePath() == null ? DFSUtilClient.EMPTY_BYTES :
+            DFSUtilClient.byteArray2bytes(entry.getSourcePath()));
+    long dirId = entry.getDirId();
+    long fileId = entry.getFileId();
+    boolean isReference = entry.isReference();
+    ByteString targetPath = getByteString(
+        entry.getTargetPath() == null ? DFSUtilClient.EMPTY_BYTES :
+            DFSUtilClient.byteArray2bytes(entry.getTargetPath()));
+    SnapshotDiffReportListingEntryProto.Builder builder =
+        SnapshotDiffReportListingEntryProto.newBuilder().setFullpath(sourcePath)
+            .setDirId(dirId).setFileId(fileId).setIsReference(isReference)
+            .setTargetPath(targetPath);
+    return builder.build();
+  }
+
+  public static SnapshotDiffReportListingProto convert(
+      SnapshotDiffReportListing report) {
+    if (report == null) {
+      return null;
+    }
+    ByteString startPath = getByteString(
+        report.getLastPath() == null ? DFSUtilClient.EMPTY_BYTES :
+            report.getLastPath());
+    List<DiffReportListingEntry> modifiedEntries = report.getModifyList();
+    List<DiffReportListingEntry> createdEntries = report.getCreateList();
+    List<DiffReportListingEntry> deletedEntries = report.getDeleteList();
+    List<SnapshotDiffReportListingEntryProto> modifiedEntryProtos =
+        new ChunkedArrayList<>();
+    for (DiffReportListingEntry entry : modifiedEntries) {
+      SnapshotDiffReportListingEntryProto entryProto = convert(entry);
+      if (entryProto != null) {
+        modifiedEntryProtos.add(entryProto);
+      }
+    }
+    List<SnapshotDiffReportListingEntryProto> createdEntryProtos =
+        new ChunkedArrayList<>();
+    for (DiffReportListingEntry entry : createdEntries) {
+      SnapshotDiffReportListingEntryProto entryProto = convert(entry);
+      if (entryProto != null) {
+        createdEntryProtos.add(entryProto);
+      }
+    }
+    List<SnapshotDiffReportListingEntryProto> deletedEntryProtos =
+        new ChunkedArrayList<>();
+    for (DiffReportListingEntry entry : deletedEntries) {
+      SnapshotDiffReportListingEntryProto entryProto = convert(entry);
+      if (entryProto != null) {
+        deletedEntryProtos.add(entryProto);
+      }
+    }
+
+    return SnapshotDiffReportListingProto.newBuilder()
+        .addAllModifiedEntries(modifiedEntryProtos)
+        .addAllCreatedEntries(createdEntryProtos)
+        .addAllDeletedEntries(deletedEntryProtos)
+        .setIsFromEarlier(report.getIsFromEarlier())
+        .setCursor(HdfsProtos.SnapshotDiffReportCursorProto.newBuilder()
+        .setStartPath(startPath)
+        .setIndex(report.getLastIndex()).build())
+        .build();
+  }
+
   public static SnapshotDiffReportProto convert(SnapshotDiffReport report) {
     if (report == null) {
       return null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index 6db6ad0..eb6da25 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -297,6 +297,16 @@ message GetSnapshotDiffReportResponseProto {
   required SnapshotDiffReportProto diffReport = 1;
 }
 
+message GetSnapshotDiffReportListingRequestProto {
+  required string snapshotRoot = 1;
+  required string fromSnapshot = 2;
+  required string toSnapshot = 3;
+  optional SnapshotDiffReportCursorProto cursor = 4;
+}
+
+message GetSnapshotDiffReportListingResponseProto {
+  required SnapshotDiffReportListingProto diffReport = 1;
+}
 message RenewLeaseRequestProto {
   required string clientName = 1;
 }
@@ -913,6 +923,8 @@ service ClientNamenodeProtocol {
       returns(DeleteSnapshotResponseProto);
   rpc getSnapshotDiffReport(GetSnapshotDiffReportRequestProto)
       returns(GetSnapshotDiffReportResponseProto);
+  rpc getSnapshotDiffReportListing(GetSnapshotDiffReportListingRequestProto)
+      returns(GetSnapshotDiffReportListingResponseProto);
   rpc isFileClosed(IsFileClosedRequestProto)
       returns(IsFileClosedResponseProto);
   rpc modifyAclEntries(ModifyAclEntriesRequestProto)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
index 953bf19..a423a4b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
@@ -529,6 +529,32 @@ message SnapshotDiffReportProto {
 }
 
 /**
+ * Snapshot diff report listing entry
+ */
+message SnapshotDiffReportListingEntryProto {
+  required bytes fullpath = 1;
+  required uint64 dirId = 2;
+  required bool isReference = 3;
+  optional bytes targetPath = 4;
+  optional uint64 fileId = 5;
+}
+
+message SnapshotDiffReportCursorProto {
+  required bytes startPath = 1;
+  required int32 index = 2 [default = -1];
+}
+/**
+ * Snapshot diff report listing
+ */
+message SnapshotDiffReportListingProto {
+  // full path of the directory where snapshots were taken
+  repeated SnapshotDiffReportListingEntryProto modifiedEntries = 1;
+  repeated SnapshotDiffReportListingEntryProto createdEntries = 2;
+  repeated SnapshotDiffReportListingEntryProto deletedEntries = 3;
+  required bool isFromEarlier = 4;
+  optional SnapshotDiffReportCursorProto cursor = 5;
+}
+/**
  * Block information
  *
  * Please be wary of adding additional fields here, since INodeFiles

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 37071b6..97b8b1a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -381,6 +381,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT_DEFAULT =
       true;
 
+  public static final String
+      DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT  =
+      "dfs.namenode.snapshotdiff.listing.limit";
+  public static final int
+      DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT_DEFAULT = 1000;
   // Whether to enable datanode's stale state detection and usage for reads
   public static final String DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY = "dfs.namenode.avoid.read.stale.datanode";
   public static final boolean DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_DEFAULT = false;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 2f9781a..3f6c3d7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -349,7 +349,8 @@ public class DFSUtil {
   public static byte[][] getPathComponents(String path) {
     // avoid intermediate split to String[]
     final byte[] bytes = string2Bytes(path);
-    return bytes2byteArray(bytes, bytes.length, (byte)Path.SEPARATOR_CHAR);
+    return DFSUtilClient
+        .bytes2byteArray(bytes, bytes.length, (byte) Path.SEPARATOR_CHAR);
   }
 
   /**
@@ -369,42 +370,9 @@ public class DFSUtil {
    * @param len the number of bytes to split
    * @param separator the delimiting byte
    */
-  public static byte[][] bytes2byteArray(byte[] bytes,
-                                         int len,
-                                         byte separator) {
-    Preconditions.checkPositionIndex(len, bytes.length);
-    if (len == 0) {
-      return new byte[][]{null};
-    }
-    // Count the splits. Omit multiple separators and the last one by
-    // peeking at prior byte.
-    int splits = 0;
-    for (int i = 1; i < len; i++) {
-      if (bytes[i-1] == separator && bytes[i] != separator) {
-        splits++;
-      }
-    }
-    if (splits == 0 && bytes[0] == separator) {
-      return new byte[][]{null};
-    }
-    splits++;
-    byte[][] result = new byte[splits][];
-    int nextIndex = 0;
-    // Build the splits.
-    for (int i = 0; i < splits; i++) {
-      int startIndex = nextIndex;
-      // find next separator in the bytes.
-      while (nextIndex < len && bytes[nextIndex] != separator) {
-        nextIndex++;
-      }
-      result[i] = (nextIndex > 0)
-          ? Arrays.copyOfRange(bytes, startIndex, nextIndex)
-          : DFSUtilClient.EMPTY_BYTES; // reuse empty bytes for root.
-      do { // skip over separators.
-        nextIndex++;
-      } while (nextIndex < len && bytes[nextIndex] == separator);
-    }
-    return result;
+  public static byte[][] bytes2byteArray(byte[] bytes, int len,
+      byte separator) {
+    return DFSUtilClient.bytes2byteArray(bytes, len, separator);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index f5bbae1..2ae41e4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
@@ -143,6 +144,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSer
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesRequestProto;
@@ -1246,6 +1249,25 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
+  public GetSnapshotDiffReportListingResponseProto getSnapshotDiffReportListing(
+      RpcController controller,
+      GetSnapshotDiffReportListingRequestProto request)
+      throws ServiceException {
+    try {
+      SnapshotDiffReportListing report = server
+          .getSnapshotDiffReportListing(request.getSnapshotRoot(),
+              request.getFromSnapshot(), request.getToSnapshot(),
+              request.getCursor().getStartPath().toByteArray(),
+              request.getCursor().getIndex());
+              //request.getStartPath(), request.getIndex());
+      return GetSnapshotDiffReportListingResponseProto.newBuilder()
+          .setDiffReport(PBHelperClient.convert(report)).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
   public IsFileClosedResponseProto isFileClosed(
       RpcController controller, IsFileClosedRequestProto request) 
       throws ServiceException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index 3bb5ca4..b5acf12 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -92,6 +92,7 @@ import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
@@ -1509,6 +1510,14 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
   }
 
   @Override // ClientProtocol
+  public SnapshotDiffReportListing getSnapshotDiffReportListing(
+      String snapshotRoot, String earlierSnapshotName, String laterSnapshotName,
+      byte[] startPath, int index) throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override // ClientProtocol
   public long addCacheDirective(CacheDirectiveInfo path,
       EnumSet<CacheFlag> flags) throws IOException {
     checkOperation(OperationCategory.WRITE, false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
index 9dd75bc..1842707 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
@@ -164,6 +165,29 @@ class FSDirSnapshotOp {
     return diffs;
   }
 
+  static SnapshotDiffReportListing getSnapshotDiffReportListing(FSDirectory fsd,
+      SnapshotManager snapshotManager, String path, String fromSnapshot,
+      String toSnapshot, byte[] startPath, int index,
+      int snapshotDiffReportLimit) throws IOException {
+    SnapshotDiffReportListing diffs;
+    final FSPermissionChecker pc = fsd.getPermissionChecker();
+    fsd.readLock();
+    try {
+      INodesInPath iip = fsd.resolvePath(pc, path, DirOp.READ);
+      if (fsd.isPermissionEnabled()) {
+        checkSubtreeReadPermission(fsd, pc, path, fromSnapshot);
+        checkSubtreeReadPermission(fsd, pc, path, toSnapshot);
+      }
+      diffs = snapshotManager
+          .diff(iip, path, fromSnapshot, toSnapshot, startPath, index,
+              snapshotDiffReportLimit);
+    } catch (Exception e) {
+      throw e;
+    } finally {
+      fsd.readUnlock();
+    }
+    return diffs;
+  }
   /** Get a collection of full snapshot paths given file and snapshot dir.
    * @param lsf a list of snapshottable features
    * @param file full path of the file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index d594f2a..d3d9cdc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -87,6 +87,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROU
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT_DEFAULT;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import static org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.*;
 
@@ -95,6 +97,8 @@ import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.server.namenode.metrics.ReplicatedBlocksMBean;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import static org.apache.hadoop.util.Time.now;
@@ -211,7 +215,6 @@ import org.apache.hadoop.hdfs.protocol.RollingUpgradeException;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@@ -426,6 +429,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   private final UserGroupInformation fsOwner;
   private final String supergroup;
   private final boolean standbyShouldCheckpoint;
+  private final int snapshotDiffReportLimit;
 
   /** Interval between each check of lease to release. */
   private final long leaseRecheckIntervalMs;
@@ -761,6 +765,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                                  DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT);
       this.isPermissionEnabled = conf.getBoolean(DFS_PERMISSIONS_ENABLED_KEY,
                                                  DFS_PERMISSIONS_ENABLED_DEFAULT);
+      this.snapshotDiffReportLimit =
+          conf.getInt(DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT,
+              DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT_DEFAULT);
+
       LOG.info("fsOwner             = " + fsOwner);
       LOG.info("supergroup          = " + supergroup);
       LOG.info("isPermissionEnabled = " + isPermissionEnabled);
@@ -6364,16 +6372,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /**
    * Get the difference between two snapshots (or between a snapshot and the
    * current status) of a snapshottable directory.
-   * 
+   *
    * @param path The full path of the snapshottable directory.
    * @param fromSnapshot Name of the snapshot to calculate the diff from. Null
    *          or empty string indicates the current tree.
    * @param toSnapshot Name of the snapshot to calculated the diff to. Null or
    *          empty string indicates the current tree.
-   * @return A report about the difference between {@code fromSnapshot} and 
-   *         {@code toSnapshot}. Modified/deleted/created/renamed files and 
-   *         directories belonging to the snapshottable directories are listed 
-   *         and labeled as M/-/+/R respectively. 
+   * @return A report about the difference between {@code fromSnapshot} and
+   *         {@code toSnapshot}. Modified/deleted/created/renamed files and
+   *         directories belonging to the snapshottable directories are listed
+   *         and labeled as M/-/+/R respectively.
    * @throws IOException
    */
   SnapshotDiffReport getSnapshotDiffReport(String path,
@@ -6403,6 +6411,63 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         toSnapshotRoot, null);
     return diffs;
   }
+
+  /**
+   * Get the difference between two snapshots (or between a snapshot and the
+   * current status) of a snapshottable directory.
+   *
+   * @param path The full path of the snapshottable directory.
+   * @param fromSnapshot Name of the snapshot to calculate the diff from. Null
+   *          or empty string indicates the current tree.
+   * @param toSnapshot Name of the snapshot to calculated the diff to. Null or
+   *          empty string indicates the current tree.
+   * @param startPath
+   *          path relative to the snapshottable root directory from where the
+   *          snapshotdiff computation needs to start across multiple rpc calls
+   * @param index
+   *           index in the created or deleted list of the directory at which
+   *           the snapshotdiff computation stopped during the last rpc call
+   *           as the no of entries exceeded the snapshotdiffentry limit. -1
+   *           indicates, the snapshotdiff compuatation needs to start right
+   *           from the startPath provided.
+   * @return A partial report about the difference between {@code fromSnapshot}
+   *         and {@code toSnapshot}. Modified/deleted/created/renamed files and
+   *         directories belonging to the snapshottable directories are listed
+   *         and labeled as M/-/+/R respectively.
+   * @throws IOException
+   */
+  SnapshotDiffReportListing getSnapshotDiffReportListing(String path,
+      String fromSnapshot, String toSnapshot, byte[] startPath, int index)
+      throws IOException {
+    final String operationName = "computeSnapshotDiff";
+    SnapshotDiffReportListing diffs = null;
+    checkOperation(OperationCategory.READ);
+    boolean success = false;
+    String fromSnapshotRoot =
+        (fromSnapshot == null || fromSnapshot.isEmpty()) ? path :
+            Snapshot.getSnapshotPath(path, fromSnapshot);
+    String toSnapshotRoot =
+        (toSnapshot == null || toSnapshot.isEmpty()) ? path :
+            Snapshot.getSnapshotPath(path, toSnapshot);
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      diffs = FSDirSnapshotOp
+          .getSnapshotDiffReportListing(dir, snapshotManager, path,
+              fromSnapshot, toSnapshot, startPath, index,
+              snapshotDiffReportLimit);
+      success = true;
+    } catch (AccessControlException ace) {
+      logAuditEvent(success, operationName, fromSnapshotRoot, toSnapshotRoot,
+          null);
+      throw ace;
+    } finally {
+      readUnlock(operationName);
+    }
+    logAuditEvent(success, operationName, fromSnapshotRoot, toSnapshotRoot,
+        null);
+    return diffs;
+  }
   
   /**
    * Delete a snapshot of a snapshottable directory

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 895e873..36d33a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -121,6 +121,7 @@ import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
@@ -1863,6 +1864,18 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
+  public SnapshotDiffReportListing getSnapshotDiffReportListing(
+      String snapshotRoot, String earlierSnapshotName, String laterSnapshotName,
+      byte[] startPath, int index) throws IOException {
+    checkNNStartup();
+    SnapshotDiffReportListing report = namesystem
+        .getSnapshotDiffReportListing(snapshotRoot, earlierSnapshotName,
+            laterSnapshotName, startPath, index);
+    metrics.incrSnapshotDiffReportOps();
+    return report;
+  }
+
+  @Override // ClientProtocol
   public long addCacheDirective(
       CacheDirectiveInfo path, EnumSet<CacheFlag> flags) throws IOException {
     checkNNStartup();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
index 076b78f..217ad01 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java
@@ -24,10 +24,12 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
+import java.util.Arrays;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
@@ -285,6 +287,54 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
   }
 
   /**
+   * Compute the difference between two snapshots (or a snapshot and the current
+   * directory) of the directory. The diff calculation can be scoped to either
+   * the snapshot root or any descendant directory under the snapshot root.
+   *
+   * @param snapshotRootDir the snapshot root directory
+   * @param snapshotDiffScopeDir the descendant directory under snapshot root
+   *          to scope the diff calculation to.
+   * @param from The name of the start point of the comparison. Null indicating
+   *          the current tree.
+   * @param to The name of the end point. Null indicating the current tree.
+   * @param startPath
+   *           path relative to the snapshottable root directory from where the
+   *           snapshotdiff computation needs to start across multiple rpc calls
+   * @param index
+   *           index in the created or deleted list of the directory at which
+   *           the snapshotdiff computation stopped during the last rpc call
+   *           as the no of entries exceeded the snapshotdiffentry limit. -1
+   *           indicates, the snapshotdiff computation needs to start right
+   *           from the startPath provided.
+   *
+   * @return The difference between the start/end points.
+   * @throws SnapshotException If there is no snapshot matching the starting
+   *           point, or if endSnapshotName is not null but cannot be identified
+   *           as a previous snapshot.
+   */
+  SnapshotDiffListingInfo computeDiff(final INodeDirectory snapshotRootDir,
+      final INodeDirectory snapshotDiffScopeDir, final String from,
+      final String to, byte[] startPath, int index,
+      int snapshotDiffReportEntriesLimit) throws SnapshotException {
+    Preconditions.checkArgument(
+        snapshotDiffScopeDir.isDescendantOfSnapshotRoot(snapshotRootDir));
+    Snapshot fromSnapshot = getSnapshotByName(snapshotRootDir, from);
+    Snapshot toSnapshot = getSnapshotByName(snapshotRootDir, to);
+    boolean toProcess = Arrays.equals(startPath, DFSUtilClient.EMPTY_BYTES);
+    byte[][] resumePath = DFSUtilClient.bytes2byteArray(startPath);
+    if (from.equals(to)) {
+      return null;
+    }
+    SnapshotDiffListingInfo diffs =
+        new SnapshotDiffListingInfo(snapshotRootDir, snapshotDiffScopeDir,
+            fromSnapshot, toSnapshot, snapshotDiffReportEntriesLimit);
+    diffs.setLastIndex(index);
+    computeDiffRecursively(snapshotDiffScopeDir, snapshotDiffScopeDir,
+        new ArrayList<byte[]>(), diffs, resumePath, 0, toProcess);
+    return diffs;
+  }
+
+  /**
    * Find the snapshot matching the given name.
    *
    * @param snapshotRoot The directory where snapshots were taken.
@@ -368,11 +418,95 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
   }
 
   /**
+   * Recursively compute the difference between snapshots under a given
+   * directory/file partially.
+   * @param snapshotDir The directory where snapshots were taken. Can be a
+   *                    snapshot root directory or any descendant directory
+   *                    under snapshot root directory.
+   * @param node The directory/file under which the diff is computed.
+   * @param parentPath Relative path (corresponding to the snapshot root) of
+   *                   the node's parent.
+   * @param diffReport data structure used to store the diff.
+   * @param resume  path from where to resume the snapshotdiff computation
+   *                    in one rpc call
+   * @param level       indicates the level of the directory tree rooted at
+   *                    snapshotRoot.
+   * @param processFlag indicates that the dir/file where the snapshotdiff
+   *                    computation has to start is processed or not.
+   */
+  private boolean computeDiffRecursively(final INodeDirectory snapshotDir,
+       INode node, List<byte[]> parentPath, SnapshotDiffListingInfo diffReport,
+       final byte[][] resume, int level, boolean processFlag) {
+    final Snapshot earlier = diffReport.getEarlier();
+    final Snapshot later = diffReport.getLater();
+    byte[][] relativePath = parentPath.toArray(new byte[parentPath.size()][]);
+    if (!processFlag && level == resume.length
+        && Arrays.equals(resume[resume.length - 1], node.getLocalNameBytes())) {
+      processFlag = true;
+    }
+
+    if (node.isDirectory()) {
+      final ChildrenDiff diff = new ChildrenDiff();
+      INodeDirectory dir = node.asDirectory();
+      if (processFlag) {
+        DirectoryWithSnapshotFeature sf = dir.getDirectoryWithSnapshotFeature();
+        if (sf != null) {
+          boolean change =
+              sf.computeDiffBetweenSnapshots(earlier, later, diff, dir);
+          if (change) {
+            if (!diffReport.addDirDiff(dir.getId(), relativePath, diff)) {
+              return false;
+            }
+          }
+        }
+      }
+
+      ReadOnlyList<INode> children = dir.getChildrenList(earlier.getId());
+      boolean iterate = false;
+      for (INode child : children) {
+        final byte[] name = child.getLocalNameBytes();
+        if (!processFlag && !iterate && !Arrays.equals(resume[level], name)) {
+          continue;
+        }
+        iterate = true;
+        level = level + 1;
+        boolean toProcess = diff.searchIndex(ListType.DELETED, name) < 0;
+        if (!toProcess && child instanceof INodeReference.WithName) {
+          byte[][] renameTargetPath = findRenameTargetPath(snapshotDir,
+              (WithName) child, Snapshot.getSnapshotId(later));
+          if (renameTargetPath != null) {
+            toProcess = true;
+          }
+        }
+        if (toProcess) {
+          parentPath.add(name);
+          processFlag = computeDiffRecursively(snapshotDir, child, parentPath,
+              diffReport, resume, level, processFlag);
+          parentPath.remove(parentPath.size() - 1);
+          if (!processFlag) {
+            return false;
+          }
+        }
+      }
+    } else if (node.isFile() && node.asFile().isWithSnapshot() && processFlag) {
+      INodeFile file = node.asFile();
+      boolean change = file.getFileWithSnapshotFeature()
+          .changedBetweenSnapshots(file, earlier, later);
+      if (change) {
+        if (!diffReport.addFileDiff(file, relativePath)) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  /**
    * We just found a deleted WithName node as the source of a rename operation.
    * However, we should include it in our snapshot diff report as rename only
    * if the rename target is also under the same snapshottable directory.
    */
-  private byte[][] findRenameTargetPath(final INodeDirectory snapshotRoot,
+  public byte[][] findRenameTargetPath(final INodeDirectory snapshotRoot,
       INodeReference.WithName wn, final int snapshotId) {
     INode inode = wn.getReferredINode();
     final LinkedList<byte[]> ancestors = Lists.newLinkedList();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotDiffListingInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotDiffListingInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotDiffListingInfo.java
new file mode 100644
index 0000000..738aa23
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotDiffListingInfo.java
@@ -0,0 +1,207 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.snapshot;
+
+import java.util.List;
+import java.util.ListIterator;
+
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing.DiffReportListingEntry;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.INodeReference;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.ChildrenDiff;
+import org.apache.hadoop.hdfs.util.Diff.ListType;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.util.ChunkedArrayList;
+
+/**
+ * A class describing the difference between snapshots of a snapshottable
+ * directory where the difference is limited by dfs.snapshotDiff-report.limit.
+ */
+
+class SnapshotDiffListingInfo {
+  private final int maxEntries;
+
+  /** The root directory of the snapshots. */
+  private final INodeDirectory snapshotRoot;
+  /**
+   *  The scope directory under which snapshot diff is calculated.
+   */
+  private final INodeDirectory snapshotDiffScopeDir;
+  /** The starting point of the difference. */
+  private final Snapshot from;
+  /** The end point of the difference. */
+  private final Snapshot to;
+
+  /** The path of the file to start for computing the snapshot diff. */
+  private byte[] lastPath = DFSUtilClient.EMPTY_BYTES;
+
+  private int lastIndex = -1;
+
+  /*
+   * A list containing all the modified entries between the given snapshots
+   * within a single rpc call.
+   */
+  private final List<DiffReportListingEntry> modifiedList =
+      new ChunkedArrayList<>();
+
+  private final List<DiffReportListingEntry> createdList =
+      new ChunkedArrayList<>();
+
+  private final List<DiffReportListingEntry> deletedList =
+      new ChunkedArrayList<>();
+
+  SnapshotDiffListingInfo(INodeDirectory snapshotRootDir,
+      INodeDirectory snapshotDiffScopeDir, Snapshot start, Snapshot end,
+      int snapshotDiffReportLimit) {
+    Preconditions.checkArgument(
+        snapshotRootDir.isSnapshottable() && snapshotDiffScopeDir
+            .isDescendantOfSnapshotRoot(snapshotRootDir));
+    this.snapshotRoot = snapshotRootDir;
+    this.snapshotDiffScopeDir = snapshotDiffScopeDir;
+    this.from = start;
+    this.to = end;
+    this.maxEntries = snapshotDiffReportLimit;
+  }
+
+  boolean addDirDiff(long dirId, byte[][] parent, ChildrenDiff diff) {
+    final Snapshot laterSnapshot = getLater();
+    if (lastIndex == -1) {
+      if (getTotalEntries() < maxEntries) {
+        modifiedList.add(new DiffReportListingEntry(
+            dirId, dirId, parent, true, null));
+      } else {
+        setLastPath(parent);
+        setLastIndex(-1);
+        return false;
+      }
+    }
+
+    if (lastIndex == -1 || lastIndex < diff.getList(ListType.CREATED).size()) {
+      ListIterator<INode> iterator = lastIndex != -1 ?
+          diff.getList(ListType.CREATED).listIterator(lastIndex)
+          : diff.getList(ListType.CREATED).listIterator();
+      while (iterator.hasNext()) {
+        if (getTotalEntries() < maxEntries) {
+          INode created = iterator.next();
+          byte[][] path = newPath(parent, created.getLocalNameBytes());
+          createdList.add(new DiffReportListingEntry(dirId, created.getId(),
+              path, created.isReference(), null));
+        } else {
+          setLastPath(parent);
+          setLastIndex(iterator.nextIndex());
+          return false;
+        }
+      }
+      setLastIndex(-1);
+    }
+
+    if (lastIndex == -1 || lastIndex >= diff.getList(ListType.CREATED).size()) {
+      int size = diff.getList(ListType.DELETED).size();
+      ListIterator<INode> iterator = lastIndex != -1 ?
+          diff.getList(ListType.DELETED).listIterator(lastIndex - size)
+          : diff.getList(ListType.DELETED).listIterator();
+      while (iterator.hasNext()) {
+        if (getTotalEntries() < maxEntries) {
+          final INode d = iterator.next();
+          byte[][] path = newPath(parent, d.getLocalNameBytes());
+          byte[][] target = findRenameTargetPath(d, laterSnapshot);
+          final DiffReportListingEntry e = target != null ?
+              new DiffReportListingEntry(dirId, d.getId(), path, true, target) :
+              new DiffReportListingEntry(dirId, d.getId(), path, false, null);
+          deletedList.add(e);
+        } else {
+          setLastPath(parent);
+          setLastIndex(size + iterator.nextIndex());
+          return false;
+        }
+      }
+      setLastIndex(-1);
+    }
+    return true;
+  }
+
+  private byte[][] findRenameTargetPath(INode deleted, Snapshot laterSnapshot) {
+    if (deleted instanceof INodeReference.WithName) {
+      return snapshotRoot.getDirectorySnapshottableFeature()
+          .findRenameTargetPath(snapshotDiffScopeDir,
+              (INodeReference.WithName) deleted,
+              Snapshot.getSnapshotId(laterSnapshot));
+    }
+    return null;
+  }
+
+  private static byte[][] newPath(byte[][] parent, byte[] name) {
+    byte[][] fullPath = new byte[parent.length + 1][];
+    System.arraycopy(parent, 0, fullPath, 0, parent.length);
+    fullPath[fullPath.length - 1] = name;
+    return fullPath;
+  }
+
+  Snapshot getEarlier() {
+    return isFromEarlier()? from: to;
+  }
+
+  Snapshot getLater() {
+    return isFromEarlier()? to: from;
+  }
+
+
+  public void setLastPath(byte[][] lastPath) {
+    this.lastPath = DFSUtilClient.byteArray2bytes(lastPath);
+  }
+
+  public void setLastIndex(int idx) {
+    this.lastIndex = idx;
+  }
+
+  boolean addFileDiff(INodeFile file, byte[][] relativePath) {
+    if (getTotalEntries() < maxEntries) {
+      modifiedList.add(new DiffReportListingEntry(file.getId(),
+          file.getId(), relativePath,false, null));
+    } else {
+      setLastPath(relativePath);
+      return false;
+    }
+    return true;
+  }
+  /** @return True if {@link #from} is earlier than {@link #to} */
+  boolean isFromEarlier() {
+    return Snapshot.ID_COMPARATOR.compare(from, to) < 0;
+  }
+
+
+  private int getTotalEntries() {
+    return createdList.size() + modifiedList.size() + deletedList.size();
+  }
+
+  /**
+   * Generate a {@link SnapshotDiffReportListing} based on detailed diff
+   * information.
+   *
+   * @return A {@link SnapshotDiffReportListing} describing the difference
+   */
+  public SnapshotDiffReportListing generateReport() {
+    return new SnapshotDiffReportListing(lastPath, modifiedList, createdList,
+        deletedList, lastIndex, isFromEarlier());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
index 58a218e..87985de 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.SnapshotInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
@@ -466,6 +467,33 @@ public class SnapshotManager implements SnapshotStatsMXBean {
     return diffs != null ? diffs.generateReport() : new SnapshotDiffReport(
         snapshotPath, from, to, Collections.<DiffReportEntry> emptyList());
   }
+
+  /**
+   * Compute the partial difference between two snapshots of a directory,
+   * or between a snapshot of the directory and its current tree.
+   */
+  public SnapshotDiffReportListing diff(final INodesInPath iip,
+      final String snapshotPath, final String from, final String to,
+      byte[] startPath, int index, int snapshotDiffReportLimit)
+      throws IOException {
+    // Find the source root directory path where the snapshots were taken.
+    // All the check for path has been included in the valueOf method.
+    INodeDirectory snapshotRootDir;
+    if (this.snapshotDiffAllowSnapRootDescendant) {
+      snapshotRootDir = getSnapshottableAncestorDir(iip);
+    } else {
+      snapshotRootDir = getSnapshottableRoot(iip);
+    }
+    Preconditions.checkNotNull(snapshotRootDir);
+    INodeDirectory snapshotDescendantDir = INodeDirectory.valueOf(
+        iip.getLastINode(), snapshotPath);
+    final SnapshotDiffListingInfo diffs =
+        snapshotRootDir.getDirectorySnapshottableFeature()
+            .computeDiff(snapshotRootDir, snapshotDescendantDir, from, to,
+                startPath, index, snapshotDiffReportLimit);
+    return diffs != null ? diffs.generateReport() :
+        new SnapshotDiffReportListing();
+  }
   
   public void clearSnapshottableDirs() {
     snapshottables.clear();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 79c2d8e..dedf987 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -4333,6 +4333,17 @@
 </property>
 
 <property>
+  <name>dfs.namenode.snapshotdiff.listing.limit</name>
+  <value>1000</value>
+  <description>
+    Limit the number of entries generated by getSnapshotDiffReportListing within
+    one rpc call to the namenode.If less or equal to zero, at most
+    DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT_DEFAULT (= 1000) will be sent
+    across to the client within one rpc call.
+  </description>
+</property>
+
+<property>
   <name>dfs.pipeline.ecn</name>
   <value>false</value>
   <description>


---------------------------------------------------------------------
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: HDFS-11576. Block recovery will fail indefinitely if recovery time > heartbeat interval. Contributed by Lukas Majercak

Posted by ae...@apache.org.
HDFS-11576. Block recovery will fail indefinitely if recovery time > heartbeat interval. Contributed by Lukas Majercak


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

Branch: refs/heads/HDFS-7240
Commit: 5304698dc8c5667c33e6ed9c4a827ef57172a723
Parents: 556aea3
Author: Chris Douglas <cd...@apache.org>
Authored: Fri Dec 1 10:29:30 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 1 10:29:30 2017 -0800

----------------------------------------------------------------------
 .../apache/hadoop/test/GenericTestUtils.java    |  10 +-
 .../server/blockmanagement/BlockManager.java    |  40 ++++++
 .../blockmanagement/PendingRecoveryBlocks.java  | 143 +++++++++++++++++++
 .../hdfs/server/namenode/FSNamesystem.java      |  40 +++---
 .../TestPendingRecoveryBlocks.java              |  87 +++++++++++
 .../hdfs/server/datanode/TestBlockRecovery.java | 108 ++++++++++++++
 .../namenode/ha/TestPipelinesFailover.java      |   5 +-
 7 files changed, 413 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5304698d/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
index 0db6c73..cdde48c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
@@ -641,10 +641,16 @@ public abstract class GenericTestUtils {
    * conditions.
    */
   public static class SleepAnswer implements Answer<Object> {
+    private final int minSleepTime;
     private final int maxSleepTime;
     private static Random r = new Random();
-    
+
     public SleepAnswer(int maxSleepTime) {
+      this(0, maxSleepTime);
+    }
+
+    public SleepAnswer(int minSleepTime, int maxSleepTime) {
+      this.minSleepTime = minSleepTime;
       this.maxSleepTime = maxSleepTime;
     }
     
@@ -652,7 +658,7 @@ public abstract class GenericTestUtils {
     public Object answer(InvocationOnMock invocation) throws Throwable {
       boolean interrupted = false;
       try {
-        Thread.sleep(r.nextInt(maxSleepTime));
+        Thread.sleep(r.nextInt(maxSleepTime) + minSleepTime);
       } catch (InterruptedException ie) {
         interrupted = true;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5304698d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 4986027..1cdb159 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -164,6 +164,8 @@ public class BlockManager implements BlockStatsMXBean {
   private static final String QUEUE_REASON_FUTURE_GENSTAMP =
     "generation stamp is in the future";
 
+  private static final long BLOCK_RECOVERY_TIMEOUT_MULTIPLIER = 30;
+
   private final Namesystem namesystem;
 
   private final BlockManagerSafeMode bmSafeMode;
@@ -353,6 +355,9 @@ public class BlockManager implements BlockStatsMXBean {
   @VisibleForTesting
   final PendingReconstructionBlocks pendingReconstruction;
 
+  /** Stores information about block recovery attempts. */
+  private final PendingRecoveryBlocks pendingRecoveryBlocks;
+
   /** The maximum number of replicas allowed for a block */
   public final short maxReplication;
   /**
@@ -549,6 +554,12 @@ public class BlockManager implements BlockStatsMXBean {
     }
     this.minReplicationToBeInMaintenance = (short)minMaintenanceR;
 
+    long heartbeatIntervalSecs = conf.getTimeDuration(
+        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
+        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS);
+    long blockRecoveryTimeout = getBlockRecoveryTimeout(heartbeatIntervalSecs);
+    pendingRecoveryBlocks = new PendingRecoveryBlocks(blockRecoveryTimeout);
+
     this.blockReportLeaseManager = new BlockReportLeaseManager(conf);
 
     bmSafeMode = new BlockManagerSafeMode(this, namesystem, haEnabled, conf);
@@ -4736,6 +4747,25 @@ public class BlockManager implements BlockStatsMXBean {
     }
   }
 
+  /**
+   * Notification of a successful block recovery.
+   * @param block for which the recovery succeeded
+   */
+  public void successfulBlockRecovery(BlockInfo block) {
+    pendingRecoveryBlocks.remove(block);
+  }
+
+  /**
+   * Checks whether a recovery attempt has been made for the given block.
+   * If so, checks whether that attempt has timed out.
+   * @param b block for which recovery is being attempted
+   * @return true if no recovery attempt has been made or
+   *         the previous attempt timed out
+   */
+  public boolean addBlockRecoveryAttempt(BlockInfo b) {
+    return pendingRecoveryBlocks.add(b);
+  }
+
   @VisibleForTesting
   public void flushBlockOps() throws IOException {
     runBlockOp(new Callable<Void>(){
@@ -4863,4 +4893,14 @@ public class BlockManager implements BlockStatsMXBean {
     }
     return i;
   }
+
+  private static long getBlockRecoveryTimeout(long heartbeatIntervalSecs) {
+    return TimeUnit.SECONDS.toMillis(heartbeatIntervalSecs *
+        BLOCK_RECOVERY_TIMEOUT_MULTIPLIER);
+  }
+
+  @VisibleForTesting
+  public void setBlockRecoveryTimeout(long blockRecoveryTimeout) {
+    pendingRecoveryBlocks.setRecoveryTimeoutInterval(blockRecoveryTimeout);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5304698d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingRecoveryBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingRecoveryBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingRecoveryBlocks.java
new file mode 100644
index 0000000..3f5f27c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingRecoveryBlocks.java
@@ -0,0 +1,143 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdfs.util.LightWeightHashSet;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * PendingRecoveryBlocks tracks recovery attempts for each block and their
+ * timeouts to ensure we do not have multiple recoveries at the same time
+ * and retry only after the timeout for a recovery has expired.
+ */
+class PendingRecoveryBlocks {
+  private static final Logger LOG = BlockManager.LOG;
+
+  /** List of recovery attempts per block and the time they expire. */
+  private final LightWeightHashSet<BlockRecoveryAttempt> recoveryTimeouts =
+      new LightWeightHashSet<>();
+
+  /** The timeout for issuing a block recovery again.
+   * (it should be larger than the time to recover a block)
+   */
+  private long recoveryTimeoutInterval;
+
+  PendingRecoveryBlocks(long timeout) {
+    this.recoveryTimeoutInterval = timeout;
+  }
+
+  /**
+   * Remove recovery attempt for the given block.
+   * @param block whose recovery attempt to remove.
+   */
+  synchronized void remove(BlockInfo block) {
+    recoveryTimeouts.remove(new BlockRecoveryAttempt(block));
+  }
+
+  /**
+   * Checks whether a recovery attempt has been made for the given block.
+   * If so, checks whether that attempt has timed out.
+   * @param block block for which recovery is being attempted
+   * @return true if no recovery attempt has been made or
+   *         the previous attempt timed out
+   */
+  synchronized boolean add(BlockInfo block) {
+    boolean added = false;
+    long curTime = getTime();
+    BlockRecoveryAttempt recoveryAttempt =
+        recoveryTimeouts.getElement(new BlockRecoveryAttempt(block));
+
+    if (recoveryAttempt == null) {
+      BlockRecoveryAttempt newAttempt = new BlockRecoveryAttempt(
+          block, curTime + recoveryTimeoutInterval);
+      added = recoveryTimeouts.add(newAttempt);
+    } else if (recoveryAttempt.hasTimedOut(curTime)) {
+      // Previous attempt timed out, reset the timeout
+      recoveryAttempt.setTimeout(curTime + recoveryTimeoutInterval);
+      added = true;
+    } else {
+      long timeoutIn = TimeUnit.MILLISECONDS.toSeconds(
+          recoveryAttempt.timeoutAt - curTime);
+      LOG.info("Block recovery attempt for " + block + " rejected, as the " +
+          "previous attempt times out in " + timeoutIn + " seconds.");
+    }
+    return added;
+  }
+
+  /**
+   * Check whether the given block is under recovery.
+   * @param b block for which to check
+   * @return true if the given block is being recovered
+   */
+  synchronized boolean isUnderRecovery(BlockInfo b) {
+    BlockRecoveryAttempt recoveryAttempt =
+        recoveryTimeouts.getElement(new BlockRecoveryAttempt(b));
+    return recoveryAttempt != null;
+  }
+
+  long getTime() {
+    return Time.monotonicNow();
+  }
+
+  @VisibleForTesting
+  synchronized void setRecoveryTimeoutInterval(long recoveryTimeoutInterval) {
+    this.recoveryTimeoutInterval = recoveryTimeoutInterval;
+  }
+
+  /**
+   * Tracks timeout for block recovery attempt of a given block.
+   */
+  private static class BlockRecoveryAttempt {
+    private final BlockInfo blockInfo;
+    private long timeoutAt;
+
+    private BlockRecoveryAttempt(BlockInfo blockInfo) {
+      this(blockInfo, 0);
+    }
+
+    BlockRecoveryAttempt(BlockInfo blockInfo, long timeoutAt) {
+      this.blockInfo = blockInfo;
+      this.timeoutAt = timeoutAt;
+    }
+
+    boolean hasTimedOut(long currentTime) {
+      return currentTime > timeoutAt;
+    }
+
+    void setTimeout(long newTimeoutAt) {
+      this.timeoutAt = newTimeoutAt;
+    }
+
+    @Override
+    public int hashCode() {
+      return blockInfo.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj instanceof BlockRecoveryAttempt) {
+        return this.blockInfo.equals(((BlockRecoveryAttempt) obj).blockInfo);
+      }
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5304698d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index d3d9cdc..6a890e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -3318,25 +3318,30 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             + "Removed empty last block and closed file " + src);
         return true;
       }
-      // start recovery of the last block for this file
-      long blockRecoveryId = nextGenerationStamp(
-          blockManager.isLegacyBlock(lastBlock));
-      lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
-      if(copyOnTruncate) {
-        lastBlock.setGenerationStamp(blockRecoveryId);
-      } else if(truncateRecovery) {
-        recoveryBlock.setGenerationStamp(blockRecoveryId);
-      }
-      uc.initializeBlockRecovery(lastBlock, blockRecoveryId, true);
-      leaseManager.renewLease(lease);
-      // Cannot close file right now, since the last block requires recovery.
-      // This may potentially cause infinite loop in lease recovery
-      // if there are no valid replicas on data-nodes.
-      NameNode.stateChangeLog.warn(
-                "DIR* NameSystem.internalReleaseLease: " +
+      // Start recovery of the last block for this file
+      // Only do so if there is no ongoing recovery for this block,
+      // or the previous recovery for this block timed out.
+      if (blockManager.addBlockRecoveryAttempt(lastBlock)) {
+        long blockRecoveryId = nextGenerationStamp(
+            blockManager.isLegacyBlock(lastBlock));
+        if(copyOnTruncate) {
+          lastBlock.setGenerationStamp(blockRecoveryId);
+        } else if(truncateRecovery) {
+          recoveryBlock.setGenerationStamp(blockRecoveryId);
+        }
+        uc.initializeBlockRecovery(lastBlock, blockRecoveryId, true);
+
+        // Cannot close file right now, since the last block requires recovery.
+        // This may potentially cause infinite loop in lease recovery
+        // if there are no valid replicas on data-nodes.
+        NameNode.stateChangeLog.warn(
+            "DIR* NameSystem.internalReleaseLease: " +
                 "File " + src + " has not been closed." +
-               " Lease recovery is in progress. " +
+                " Lease recovery is in progress. " +
                 "RecoveryId = " + blockRecoveryId + " for block " + lastBlock);
+      }
+      lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
+      leaseManager.renewLease(lease);
       break;
     }
     return false;
@@ -3604,6 +3609,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         // If this commit does not want to close the file, persist blocks
         FSDirWriteFileOp.persistBlocks(dir, src, iFile, false);
       }
+      blockManager.successfulBlockRecovery(storedBlock);
     } finally {
       writeUnlock("commitBlockSynchronization");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5304698d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingRecoveryBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingRecoveryBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingRecoveryBlocks.java
new file mode 100644
index 0000000..baad89f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingRecoveryBlocks.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * This class contains unit tests for PendingRecoveryBlocks.java functionality.
+ */
+public class TestPendingRecoveryBlocks {
+
+  private PendingRecoveryBlocks pendingRecoveryBlocks;
+  private final long recoveryTimeout = 1000L;
+
+  private final BlockInfo blk1 = getBlock(1);
+  private final BlockInfo blk2 = getBlock(2);
+  private final BlockInfo blk3 = getBlock(3);
+
+  @Before
+  public void setUp() {
+    pendingRecoveryBlocks =
+        Mockito.spy(new PendingRecoveryBlocks(recoveryTimeout));
+  }
+
+  BlockInfo getBlock(long blockId) {
+    return new BlockInfoContiguous(new Block(blockId), (short) 0);
+  }
+
+  @Test
+  public void testAddDifferentBlocks() {
+    assertTrue(pendingRecoveryBlocks.add(blk1));
+    assertTrue(pendingRecoveryBlocks.isUnderRecovery(blk1));
+    assertTrue(pendingRecoveryBlocks.add(blk2));
+    assertTrue(pendingRecoveryBlocks.isUnderRecovery(blk2));
+    assertTrue(pendingRecoveryBlocks.add(blk3));
+    assertTrue(pendingRecoveryBlocks.isUnderRecovery(blk3));
+  }
+
+  @Test
+  public void testAddAndRemoveBlocks() {
+    // Add blocks
+    assertTrue(pendingRecoveryBlocks.add(blk1));
+    assertTrue(pendingRecoveryBlocks.add(blk2));
+
+    // Remove blk1
+    pendingRecoveryBlocks.remove(blk1);
+
+    // Adding back blk1 should succeed
+    assertTrue(pendingRecoveryBlocks.add(blk1));
+  }
+
+  @Test
+  public void testAddBlockWithPreviousRecoveryTimedOut() {
+    // Add blk
+    Mockito.doReturn(0L).when(pendingRecoveryBlocks).getTime();
+    assertTrue(pendingRecoveryBlocks.add(blk1));
+
+    // Should fail, has not timed out yet
+    Mockito.doReturn(recoveryTimeout / 2).when(pendingRecoveryBlocks).getTime();
+    assertFalse(pendingRecoveryBlocks.add(blk1));
+
+    // Should succeed after timing out
+    Mockito.doReturn(recoveryTimeout * 2).when(pendingRecoveryBlocks).getTime();
+    assertTrue(pendingRecoveryBlocks.add(blk1));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5304698d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
index 311d5a6..208447d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
@@ -18,7 +18,10 @@
 
 package org.apache.hadoop.hdfs.server.datanode;
 
+import org.apache.hadoop.hdfs.AppendTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
+
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
@@ -43,6 +46,7 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Random;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
@@ -94,6 +98,7 @@ import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.GenericTestUtils.SleepAnswer;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
@@ -1035,4 +1040,107 @@ public class TestBlockRecovery {
       Assert.fail("Thread failure: " + failureReason);
     }
   }
+
+  /**
+   * Test for block recovery taking longer than the heartbeat interval.
+   */
+  @Test(timeout = 300000L)
+  public void testRecoverySlowerThanHeartbeat() throws Exception {
+    tearDown(); // Stop the Mocked DN started in startup()
+
+    SleepAnswer delayer = new SleepAnswer(3000, 6000);
+    testRecoveryWithDatanodeDelayed(delayer);
+  }
+
+  /**
+   * Test for block recovery timeout. All recovery attempts will be delayed
+   * and the first attempt will be lost to trigger recovery timeout and retry.
+   */
+  @Test(timeout = 300000L)
+  public void testRecoveryTimeout() throws Exception {
+    tearDown(); // Stop the Mocked DN started in startup()
+    final Random r = new Random();
+
+    // Make sure first commitBlockSynchronization call from the DN gets lost
+    // for the recovery timeout to expire and new recovery attempt
+    // to be started.
+    SleepAnswer delayer = new SleepAnswer(3000) {
+      private final AtomicBoolean callRealMethod = new AtomicBoolean();
+
+      @Override
+      public Object answer(InvocationOnMock invocation) throws Throwable {
+        boolean interrupted = false;
+        try {
+          Thread.sleep(r.nextInt(3000) + 6000);
+        } catch (InterruptedException ie) {
+          interrupted = true;
+        }
+        try {
+          if (callRealMethod.get()) {
+            return invocation.callRealMethod();
+          }
+          callRealMethod.set(true);
+          return null;
+        } finally {
+          if (interrupted) {
+            Thread.currentThread().interrupt();
+          }
+        }
+      }
+    };
+    testRecoveryWithDatanodeDelayed(delayer);
+  }
+
+  private void testRecoveryWithDatanodeDelayed(
+      GenericTestUtils.SleepAnswer recoveryDelayer) throws Exception {
+    Configuration configuration = new HdfsConfiguration();
+    configuration.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    MiniDFSCluster cluster = null;
+
+    try {
+      cluster = new MiniDFSCluster.Builder(configuration)
+          .numDataNodes(2).build();
+      cluster.waitActive();
+      final FSNamesystem ns = cluster.getNamesystem();
+      final NameNode nn = cluster.getNameNode();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      ns.getBlockManager().setBlockRecoveryTimeout(
+          TimeUnit.SECONDS.toMillis(10));
+
+      // Create a file and never close the output stream to trigger recovery
+      FSDataOutputStream out = dfs.create(new Path("/testSlowRecovery"),
+          (short) 2);
+      out.write(AppendTestUtil.randomBytes(0, 4096));
+      out.hsync();
+
+      List<DataNode> dataNodes = cluster.getDataNodes();
+      for (DataNode datanode : dataNodes) {
+        DatanodeProtocolClientSideTranslatorPB nnSpy =
+            InternalDataNodeTestUtils.spyOnBposToNN(datanode, nn);
+
+        Mockito.doAnswer(recoveryDelayer).when(nnSpy).
+            commitBlockSynchronization(
+                Mockito.any(ExtendedBlock.class), Mockito.anyInt(),
+                Mockito.anyLong(), Mockito.anyBoolean(),
+                Mockito.anyBoolean(), Mockito.anyObject(),
+                Mockito.anyObject());
+      }
+
+      // Make sure hard lease expires to trigger replica recovery
+      cluster.setLeasePeriod(100L, 100L);
+
+      // Wait for recovery to succeed
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          return ns.getCompleteBlocksTotal() > 0;
+        }
+      }, 300, 300000);
+
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5304698d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
index dc7f47a..a565578 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.Log;
@@ -278,12 +279,14 @@ public class TestPipelinesFailover {
     // Disable permissions so that another user can recover the lease.
     conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, false);
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    
+
     FSDataOutputStream stm = null;
     final MiniDFSCluster cluster = newMiniCluster(conf, 3);
     try {
       cluster.waitActive();
       cluster.transitionToActive(0);
+      cluster.getNamesystem().getBlockManager().setBlockRecoveryTimeout(
+          TimeUnit.SECONDS.toMillis(1));
       Thread.sleep(500);
 
       LOG.info("Starting with NN 0 active");


---------------------------------------------------------------------
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: YARN-7573. Gpu Information page could be empty for nodes without GPU. (Sunil G via wangda)

Posted by ae...@apache.org.
YARN-7573. Gpu Information page could be empty for nodes without GPU. (Sunil G via wangda)

Change-Id: I7f614e5a589a09ce4e4286c84b706e05c29abd14


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

Branch: refs/heads/HDFS-7240
Commit: c9a54aab6b1ad91b14de934178018d8e7eecd001
Parents: 333ef30
Author: Wangda Tan <wa...@apache.org>
Authored: Wed Nov 29 17:43:37 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed Nov 29 17:46:16 2017 -0800

----------------------------------------------------------------------
 .../hadoop/yarn/server/nodemanager/webapp/NMWebServices.java   | 4 +---
 .../hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-node.js  | 6 ++++--
 .../main/webapp/app/templates/components/node-menu-panel.hbs   | 2 +-
 .../src/main/webapp/app/templates/yarn-node-apps.hbs           | 2 +-
 .../src/main/webapp/app/templates/yarn-node-containers.hbs     | 2 +-
 .../src/main/webapp/app/templates/yarn-node/yarn-nm-gpu.hbs    | 4 ++++
 6 files changed, 12 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9a54aab/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 937a8cd..9157374 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
@@ -519,9 +519,7 @@ public class NMWebServices {
       }
     }
 
-    throw new YarnException(
-        "Could not get detailed resource information for given resource-name="
-            + resourceName);
+    return new NMResourceInfo();
   }
 
   private long parseLongParam(String bytes) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9a54aab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-node.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-node.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-node.js
index da5ba01..3b2385f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-node.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-rm-node.js
@@ -97,7 +97,9 @@ export default DS.Model.extend({
     var used = 0;
     var ri;
 
-    var resourceInformations = this.get("usedResource").resourcesInformations;
+    const usedResource = this.get("usedResource");
+    const availableResource = this.get("availableResource");
+    var resourceInformations = usedResource ? usedResource.resourcesInformations : [];
     for (var i = 0; i < resourceInformations.length; i++) {
       ri = resourceInformations[i];
       if (ri.name === "yarn.io/gpu") {
@@ -106,7 +108,7 @@ export default DS.Model.extend({
     }
 
     var available = 0;
-    resourceInformations = this.get("availableResource").resourcesInformations;
+    resourceInformations = availableResource ? availableResource.resourcesInformations : [];
     for (i = 0; i < resourceInformations.length; i++) {
       ri = resourceInformations[i];
       if (ri.name === "yarn.io/gpu") {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9a54aab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/node-menu-panel.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/node-menu-panel.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/node-menu-panel.hbs
index fffae30..966e408 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/node-menu-panel.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/node-menu-panel.hbs
@@ -36,7 +36,7 @@
               {{#link-to 'yarn-node-containers' nodeId nodeAddr}}List of Containers
               {{/link-to}}
             {{/link-to}}
-            {{#if nmGpuInfo}}
+            {{#if (and nmGpuInfo nmGpuInfo.info.totalGpuDevices)}}
               {{#link-to 'yarn-node.yarn-nm-gpu' tagName="li"}}
                 {{#link-to 'yarn-node.yarn-nm-gpu' nodeId nodeAddr }}GPU Information
                 {{/link-to}}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9a54aab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-node-apps.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-node-apps.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-node-apps.hbs
index 52f0c86..919e54d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-node-apps.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-node-apps.hbs
@@ -20,7 +20,7 @@
 
 <div class="col-md-12 container-fluid">
   <div class="row">
-    {{node-menu-panel path="yarn-node-apps" nodeAddr=model.nodeInfo.addr nodeId=model.nodeInfo.id}}
+    {{node-menu-panel path="yarn-node-apps" nodeAddr=model.nodeInfo.addr nodeId=model.nodeInfo.id nmGpuInfo=model.nmGpuInfo}}
     {{#if model.apps}}
     <div class="col-md-10 container-fluid">
       <table id="node-apps-table" class="display table table-striped table-bordered" cellspacing="0" width="100%">

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9a54aab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-node-containers.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-node-containers.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-node-containers.hbs
index f520c46..1f31272 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-node-containers.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-node-containers.hbs
@@ -20,7 +20,7 @@
 
 <div class="col-md-12 container-fluid">
   <div class="row">
-    {{node-menu-panel path="yarn-node-containers" nodeAddr=model.nodeInfo.addr nodeId=model.nodeInfo.id}}
+    {{node-menu-panel path="yarn-node-containers" nodeAddr=model.nodeInfo.addr nodeId=model.nodeInfo.id nmGpuInfo=model.nmGpuInfo}}
     {{#if model.containers}}
     <div class="col-md-10 container-fluid">
       <table id="node-containers-table" class="display table table-striped table-bordered" cellspacing="0" width="100%">

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9a54aab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-node/yarn-nm-gpu.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-node/yarn-nm-gpu.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-node/yarn-nm-gpu.hbs
index 55840ad..0464cc8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-node/yarn-nm-gpu.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-node/yarn-nm-gpu.hbs
@@ -23,6 +23,7 @@
 
     {{node-menu-panel path="yarn-node" nodeId=model.rmNode.id
                       nodeAddr=model.node.id nmGpuInfo=model.nmGpuInfo}}
+    {{#if model.nmGpuInfo.info.totalGpuDevices}}
 
     <div class="col-md-10 container-fluid">
       <div class="panel panel-default">
@@ -49,5 +50,8 @@
         {{yarn-nm-gpu-info gpu=gpu}}
       {{/each}}
     </div>
+    {{else}}
+      <h4 align = "center">No GPUs are found on this node.</h4>
+    {{/if}}
   </div>
 </div>
\ 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


[33/50] [abbrv] hadoop git commit: YARN-6669. Implemented Kerberos security for YARN service framework. (Contributed by Jian He)

Posted by ae...@apache.org.
YARN-6669.  Implemented Kerberos security for YARN service framework.  (Contributed by Jian He)


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

Branch: refs/heads/HDFS-7240
Commit: d30d57828fddaa8667de49af879cde999907c7f6
Parents: 404eab4
Author: Eric Yang <ey...@apache.org>
Authored: Mon Dec 4 15:11:00 2017 -0500
Committer: Eric Yang <ey...@apache.org>
Committed: Mon Dec 4 15:11:00 2017 -0500

----------------------------------------------------------------------
 .../hadoop/yarn/service/webapp/ApiServer.java   |   1 +
 ...RN-Simplified-V1-API-Layer-For-Services.yaml |  16 +
 .../dev-support/findbugs-exclude.xml            |   5 +-
 .../yarn/service/ClientAMPolicyProvider.java    |  39 ++
 .../yarn/service/ClientAMSecurityInfo.java      |  62 ++++
 .../hadoop/yarn/service/ClientAMService.java    |   9 +
 .../hadoop/yarn/service/ServiceContext.java     |   8 +
 .../hadoop/yarn/service/ServiceMaster.java      | 140 ++++++-
 .../hadoop/yarn/service/ServiceScheduler.java   |  30 +-
 .../service/api/records/KerberosPrincipal.java  | 146 ++++++++
 .../yarn/service/api/records/Service.java       |  23 ++
 .../yarn/service/client/ServiceClient.java      | 174 ++++++---
 .../yarn/service/component/Component.java       |   2 +-
 .../yarn/service/conf/YarnServiceConf.java      |   7 -
 .../yarn/service/conf/YarnServiceConstants.java |   3 +
 .../containerlaunch/AbstractLauncher.java       |  39 +-
 .../containerlaunch/ContainerLaunchService.java |  10 +-
 .../containerlaunch/CredentialUtils.java        | 319 ----------------
 .../hadoop/yarn/service/package-info.java       |  24 ++
 .../yarn/service/provider/ProviderUtils.java    |  53 +--
 .../yarn/service/utils/ServiceApiUtil.java      |  15 +
 .../hadoop/yarn/service/utils/ServiceUtils.java |  31 +-
 .../org.apache.hadoop.security.SecurityInfo     |  14 +
 .../client/api/RegistryOperationsFactory.java   |  21 ++
 .../registry/client/impl/zk/CuratorService.java |   8 +-
 .../client/impl/zk/RegistrySecurity.java        |  96 ++++-
 .../hadoop/registry/server/dns/RegistryDNS.java |   4 +
 .../RMRegistryOperationsService.java            | 246 -------------
 .../services/DeleteCompletionCallback.java      |   3 +-
 .../hadoop/registry/AbstractRegistryTest.java   |  15 +-
 .../integration/TestRegistryRMOperations.java   | 369 -------------------
 .../secure/TestSecureRMRegistryOperations.java  | 348 -----------------
 .../site/markdown/yarn-service/QuickStart.md    |  20 +-
 .../markdown/yarn-service/YarnServiceAPI.md     |  11 +-
 34 files changed, 844 insertions(+), 1467 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
index 1bb6c93..34ab8f0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
@@ -173,6 +173,7 @@ public class ApiServer {
       return Response.status(Status.BAD_REQUEST).entity(serviceStatus)
           .build();
     } catch (Exception e) {
+      LOG.error("Fail to stop service:", e);
       ServiceStatus serviceStatus = new ServiceStatus();
       serviceStatus.setDiagnostics(e.getMessage());
       return Response.status(Status.INTERNAL_SERVER_ERROR)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml
index 088b50c..979883c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml
@@ -244,6 +244,10 @@ definitions:
       queue:
         type: string
         description: The YARN queue that this service should be submitted to.
+      kerberos_principal:
+        description: The Kerberos Principal of the service
+        $ref: '#/definitions/KerberosPrincipal'
+
   Resource:
     description:
       Resource determines the amount of resources (vcores, memory, network, etc.) usable by a container. This field determines the resource to be applied for all the containers of a component or service. The resource specified at the service (or global) level can be overriden at the component level. Only one of profile OR cpu & memory are expected. It raises a validation exception otherwise.
@@ -469,3 +473,15 @@ definitions:
         type: integer
         format: int32
         description: An error code specific to a scenario which service owners should be able to use to understand the failure in addition to the diagnostic information.
+  KerberosPrincipal:
+    description: The kerberos principal info of the user who launches the service.
+    properties:
+      principal_name:
+        type: string
+        description: The principal name of the user who launches the service.
+      keytab:
+        type: string
+        description: |
+          The URI of the kerberos keytab. It supports two modes:
+          URI starts with "hdfs://": A path on hdfs where the keytab is stored. The keytab will be localized by YARN to each host.
+          URI starts with "file://": A path on the local host where the keytab is stored. It is assumed that the keytabs are pre-installed by admins before AM launches.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/dev-support/findbugs-exclude.xml
index 2814cca..80c04c8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/dev-support/findbugs-exclude.xml
@@ -44,5 +44,8 @@
         <Field name="registryClient" />
         <Bug pattern="IS2_INCONSISTENT_SYNC"/>
     </Match>
-
+    <Match>
+        <Class name="org.apache.hadoop.yarn.service.ClientAMPolicyProvider"/>
+        <Bug pattern="EI_EXPOSE_REP"/>
+    </Match>
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMPolicyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMPolicyProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMPolicyProvider.java
new file mode 100644
index 0000000..365df0f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMPolicyProvider.java
@@ -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.
+ */
+
+package org.apache.hadoop.yarn.service;
+
+import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.apache.hadoop.security.authorize.Service;
+
+/**
+ * PolicyProvider for Client to Service AM protocol.
+ */
+public class ClientAMPolicyProvider extends PolicyProvider {
+
+  private static final Service[] CLIENT_AM_SERVICE =
+      new Service[]{
+          new Service(
+              "security.yarn-service.client-am-protocol.acl",
+              ClientAMProtocol.class)};
+
+  @Override
+  public Service[] getServices() {
+    return CLIENT_AM_SERVICE;
+  };
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMSecurityInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMSecurityInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMSecurityInfo.java
new file mode 100644
index 0000000..e19284b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMSecurityInfo.java
@@ -0,0 +1,62 @@
+/**
+ * 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.service;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.KerberosInfo;
+import org.apache.hadoop.security.SecurityInfo;
+import org.apache.hadoop.security.token.TokenInfo;
+import org.apache.hadoop.yarn.service.conf.YarnServiceConstants;
+import org.apache.hadoop.yarn.service.impl.pb.service.ClientAMProtocolPB;
+
+import java.lang.annotation.Annotation;
+
+/**
+ * Security Info for Client to Service AM protocol.
+ */
+public class ClientAMSecurityInfo extends SecurityInfo{
+  @Override
+  public KerberosInfo getKerberosInfo(Class<?> protocol, Configuration conf) {
+    if (!protocol.equals(ClientAMProtocolPB.class)) {
+      return null;
+    }
+    return new KerberosInfo() {
+
+      @Override
+      public Class<? extends Annotation> annotationType() {
+        return null;
+      }
+
+      @Override
+      public String serverPrincipal() {
+        return YarnServiceConstants.PRINCIPAL;
+      }
+
+      @Override
+      public String clientPrincipal() {
+        return null;
+      }
+    };
+  }
+
+  @Override
+  public TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMService.java
index 8e4c34d..94dd8d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMService.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.service;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.service.AbstractService;
@@ -65,6 +66,14 @@ public class ClientAMService extends AbstractService
     InetSocketAddress address = new InetSocketAddress(0);
     server = rpc.getServer(ClientAMProtocol.class, this, address, conf,
         context.secretManager, 1);
+
+    // Enable service authorization?
+    if (conf.getBoolean(
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
+        false)) {
+      this.server.refreshServiceAcl(getConfig(), new ClientAMPolicyProvider());
+    }
+
     server.start();
 
     String nodeHostString =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceContext.java
index 94dbc6e..cd41ab7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceContext.java
@@ -25,6 +25,8 @@ import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.api.records.ConfigFile;
 import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
 
+import java.nio.ByteBuffer;
+
 public class ServiceContext {
   public Service service = null;
   public SliderFileSystem fs;
@@ -34,6 +36,12 @@ public class ServiceContext {
   public ServiceScheduler scheduler;
   public ClientToAMTokenSecretManager secretManager;
   public ClientAMService clientAMService;
+  // tokens used for container launch
+  public ByteBuffer tokens;
+  // AM keytab principal
+  public String principal;
+  // AM keytab location
+  public String keytab;
 
   public ServiceContext() {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java
index b0b4f06..1283604 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java
@@ -20,33 +20,49 @@ package org.apache.hadoop.yarn.service;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Options;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+
+import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.ShutdownHookManager;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
+import org.apache.hadoop.yarn.service.exceptions.BadClusterStateException;
 import org.apache.hadoop.yarn.service.monitor.ServiceMonitor;
 import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
-import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
 import org.apache.hadoop.yarn.service.utils.ServiceUtils;
-import org.apache.hadoop.yarn.service.exceptions.BadClusterStateException;
+import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
 import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedExceptionAction;
+import java.util.Iterator;
 import java.util.Map;
 
+import static org.apache.hadoop.yarn.service.conf.YarnServiceConstants.KEYTAB_LOCATION;
+
 public class ServiceMaster extends CompositeService {
 
   private static final Logger LOG =
@@ -63,13 +79,7 @@ public class ServiceMaster extends CompositeService {
 
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
-    //TODO Deprecate slider conf, make sure works with yarn conf
     printSystemEnv();
-    if (UserGroupInformation.isSecurityEnabled()) {
-      UserGroupInformation.setConfiguration(conf);
-    }
-    LOG.info("Login user is {}", UserGroupInformation.getLoginUser());
-
     context = new ServiceContext();
     Path appDir = getAppDir();
     context.serviceHdfsDir = appDir.toString();
@@ -78,6 +88,10 @@ public class ServiceMaster extends CompositeService {
     fs.setAppDir(appDir);
     loadApplicationJson(context, fs);
 
+    if (UserGroupInformation.isSecurityEnabled()) {
+      context.tokens = recordTokensForContainers();
+      doSecureLogin();
+    }
     // Take yarn config from YarnFile and merge them into YarnConfiguration
     for (Map.Entry<String, String> entry : context.service
         .getConfiguration().getProperties().entrySet()) {
@@ -111,6 +125,100 @@ public class ServiceMaster extends CompositeService {
     super.serviceInit(conf);
   }
 
+  // Record the tokens and use them for launching containers.
+  // e.g. localization requires the hdfs delegation tokens
+  private ByteBuffer recordTokensForContainers() throws IOException {
+    Credentials copy = new Credentials(UserGroupInformation.getCurrentUser()
+        .getCredentials());
+    DataOutputBuffer dob = new DataOutputBuffer();
+    try {
+      copy.writeTokenStorageToStream(dob);
+    } finally {
+      dob.close();
+    }
+    // Now remove the AM->RM token so that task containers cannot access it.
+    Iterator<Token<?>> iter = copy.getAllTokens().iterator();
+    while (iter.hasNext()) {
+      Token<?> token = iter.next();
+      LOG.info(token.toString());
+      if (token.getKind().equals(AMRMTokenIdentifier.KIND_NAME)) {
+        iter.remove();
+      }
+    }
+    return ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+  }
+
+  // 1. First try to use user specified keytabs
+  // 2. If not specified, then try to use pre-installed keytab at localhost
+  // 3. strip off hdfs delegation tokens to ensure use keytab to talk to hdfs
+  private void doSecureLogin()
+      throws IOException, URISyntaxException {
+    // read the localized keytab specified by user
+    File keytab = new File(String.format(KEYTAB_LOCATION,
+        context.service.getName()));
+    if (!keytab.exists()) {
+      LOG.info("No keytab localized at " + keytab);
+      // Check if there exists a pre-installed keytab at host
+      String preInstalledKeytab = context.service.getKerberosPrincipal()
+          .getKeytab();
+      if (!StringUtils.isEmpty(preInstalledKeytab)) {
+        URI uri = new URI(preInstalledKeytab);
+        if (uri.getScheme().equals("file")) {
+          keytab = new File(uri);
+          LOG.info("Using pre-installed keytab from localhost: " +
+              preInstalledKeytab);
+        }
+      }
+    }
+    if (!keytab.exists()) {
+      LOG.info("No keytab exists: " + keytab);
+      return;
+    }
+    String principal = context.service.getKerberosPrincipal()
+        .getPrincipalName();
+    if (StringUtils.isEmpty((principal))) {
+      principal = UserGroupInformation.getLoginUser().getShortUserName();
+      LOG.info("No principal name specified.  Will use AM " +
+          "login identity {} to attempt keytab-based login", principal);
+    }
+
+    Credentials credentials = UserGroupInformation.getCurrentUser()
+        .getCredentials();
+    LOG.info("User before logged in is: " + UserGroupInformation
+        .getCurrentUser());
+    String principalName = SecurityUtil.getServerPrincipal(principal,
+        ServiceUtils.getLocalHostName(getConfig()));
+    UserGroupInformation.loginUserFromKeytab(principalName,
+        keytab.getAbsolutePath());
+    // add back the credentials
+    UserGroupInformation.getCurrentUser().addCredentials(credentials);
+    LOG.info("User after logged in is: " + UserGroupInformation
+        .getCurrentUser());
+    context.principal = principalName;
+    context.keytab = keytab.getAbsolutePath();
+    removeHdfsDelegationToken(UserGroupInformation.getLoginUser());
+  }
+
+  // Remove HDFS delegation token from login user and ensure AM to use keytab
+  // to talk to hdfs
+  private static void removeHdfsDelegationToken(UserGroupInformation user) {
+    if (!user.isFromKeytab()) {
+      LOG.error("AM is not holding on a keytab in a secure deployment:" +
+          " service will fail when tokens expire");
+    }
+    Credentials credentials = user.getCredentials();
+    Iterator<Token<? extends TokenIdentifier>> iter =
+        credentials.getAllTokens().iterator();
+    while (iter.hasNext()) {
+      Token<? extends TokenIdentifier> token = iter.next();
+      if (token.getKind().equals(
+          DelegationTokenIdentifier.HDFS_DELEGATION_KIND)) {
+        LOG.info("Remove HDFS delegation token {}.", token);
+        iter.remove();
+      }
+    }
+  }
+
   protected ContainerId getAMContainerId() throws BadClusterStateException {
     return ContainerId.fromString(ServiceUtils.mandatoryEnvVariable(
         ApplicationConstants.Environment.CONTAINER_ID.name()));
@@ -133,6 +241,17 @@ public class ServiceMaster extends CompositeService {
   }
 
   @Override
+  protected void serviceStart() throws Exception {
+    LOG.info("Starting service as user " + UserGroupInformation
+        .getCurrentUser());
+    UserGroupInformation.getLoginUser().doAs(
+        (PrivilegedExceptionAction<Void>) () -> {
+          super.serviceStart();
+          return null;
+        }
+    );
+  }
+  @Override
   protected void serviceStop() throws Exception {
     LOG.info("Stopping app master");
     super.serviceStop();
@@ -146,7 +265,8 @@ public class ServiceMaster extends CompositeService {
 
   public static void main(String[] args) throws Exception {
     Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());
-    StringUtils.startupShutdownMessage(ServiceMaster.class, args, LOG);
+    org.apache.hadoop.util.StringUtils
+        .startupShutdownMessage(ServiceMaster.class, args, LOG);
     try {
       ServiceMaster serviceMaster = new ServiceMaster("Service Master");
       ShutdownHookManager.get()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
index 6bc5673..bea31cf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
@@ -22,6 +22,7 @@ import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -34,6 +35,7 @@ import org.apache.hadoop.registry.client.binding.RegistryUtils;
 import org.apache.hadoop.registry.client.types.ServiceRecord;
 import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
 import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
+import org.apache.hadoop.security.HadoopKerberosName;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
@@ -142,11 +144,29 @@ public class ServiceScheduler extends CompositeService {
   }
 
   public void buildInstance(ServiceContext context, Configuration configuration)
-      throws YarnException {
+      throws YarnException, IOException {
     app = context.service;
     executorService = Executors.newScheduledThreadPool(10);
-    RegistryOperations registryClient = RegistryOperationsFactory
-        .createInstance("ServiceScheduler", configuration);
+    RegistryOperations registryClient = null;
+    if (UserGroupInformation.isSecurityEnabled() &&
+        !StringUtils.isEmpty(context.principal)
+        && !StringUtils.isEmpty(context.keytab)) {
+      Configuration conf = getConfig();
+      // Only take the first section of the principal
+      // e.g. hdfs-demo@EXAMPLE.COM will take hdfs-demo
+      // This is because somehow zookeeper client only uses the first section
+      // for acl validations.
+      String username = new HadoopKerberosName(context.principal.trim())
+          .getServiceName();
+      LOG.info("Set registry user accounts: sasl:" + username);
+      conf.set(KEY_REGISTRY_USER_ACCOUNTS, "sasl:" + username);
+      registryClient = RegistryOperationsFactory
+          .createKerberosInstance(conf,
+              "Client", context.principal, context.keytab);
+    } else {
+      registryClient = RegistryOperationsFactory
+          .createInstance("ServiceScheduler", configuration);
+    }
     addIfService(registryClient);
     yarnRegistryOperations =
         createYarnRegistryOperations(context, registryClient);
@@ -171,7 +191,7 @@ public class ServiceScheduler extends CompositeService {
     dispatcher.setDrainEventsOnStop();
     addIfService(dispatcher);
 
-    containerLaunchService = new ContainerLaunchService(context.fs);
+    containerLaunchService = new ContainerLaunchService(context);
     addService(containerLaunchService);
 
     if (YarnConfiguration.timelineServiceV2Enabled(configuration)) {
@@ -408,7 +428,7 @@ public class ServiceScheduler extends CompositeService {
           }
         } catch (IOException e) {
           LOG.error(
-              "Failed to register app " + app.getName() + " in registry");
+              "Failed to register app " + app.getName() + " in registry", e);
         }
       }
     });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/KerberosPrincipal.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/KerberosPrincipal.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/KerberosPrincipal.java
new file mode 100644
index 0000000..e38fdb5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/KerberosPrincipal.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.service.api.records;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import javax.xml.bind.annotation.XmlElement;
+import java.io.Serializable;
+import java.util.Objects;
+
+/**
+ * The kerberos principal of the service.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+@ApiModel(description = "The kerberos principal of the service.")
+@javax.annotation.Generated(value = "io.swagger.codegen.languages" +
+    ".JavaClientCodegen", date = "2017-11-20T11:29:11.785-08:00")
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class KerberosPrincipal implements Serializable {
+  private static final long serialVersionUID = -6431667195287650037L;
+
+  @JsonProperty("principal_name")
+  @XmlElement(name = "principal_name")
+  private String principalName = null;
+
+  @JsonProperty("keytab")
+  @XmlElement(name = "keytab")
+  private String keytab = null;
+
+  public KerberosPrincipal principalName(String principalName) {
+    this.principalName = principalName;
+    return this;
+  }
+
+  /**
+   * The principal name of the service.
+   *
+   * @return principalName
+   **/
+  @ApiModelProperty(value = "The principal name of the service.")
+  public String getPrincipalName() {
+    return principalName;
+  }
+
+  public void setPrincipalName(String principalName) {
+    this.principalName = principalName;
+  }
+
+  public KerberosPrincipal keytab(String keytab) {
+    this.keytab = keytab;
+    return this;
+  }
+
+  /**
+   * The URI of the kerberos keytab. It supports two schemes \&quot;
+   * hdfs\&quot; and \&quot;file\&quot;. If the URI starts with \&quot;
+   * hdfs://\&quot; scheme, it indicates the path on hdfs where the keytab is
+   * stored. The keytab will be localized by YARN and made available to AM in
+   * its local directory. If the URI starts with \&quot;file://\&quot;
+   * scheme, it indicates a path on the local host presumbaly installed by
+   * admins upfront.
+   *
+   * @return keytab
+   **/
+  @ApiModelProperty(value = "The URI of the kerberos keytab. It supports two " +
+      "schemes \"hdfs\" and \"file\". If the URI starts with \"hdfs://\" " +
+      "scheme, it indicates the path on hdfs where the keytab is stored. The " +
+      "keytab will be localized by YARN and made available to AM in its local" +
+      " directory. If the URI starts with \"file://\" scheme, it indicates a " +
+      "path on the local host where the keytab is presumbaly installed by " +
+      "admins upfront. ")
+  public String getKeytab() {
+    return keytab;
+  }
+
+  public void setKeytab(String keytab) {
+    this.keytab = keytab;
+  }
+
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    KerberosPrincipal kerberosPrincipal = (KerberosPrincipal) o;
+    return Objects.equals(this.principalName, kerberosPrincipal
+        .principalName) &&
+        Objects.equals(this.keytab, kerberosPrincipal.keytab);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(principalName, keytab);
+  }
+
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class KerberosPrincipal {\n");
+
+    sb.append("    principalName: ").append(toIndentedString(principalName))
+        .append("\n");
+    sb.append("    keytab: ").append(toIndentedString(keytab)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Service.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Service.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Service.java
index 8045822..392b71e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Service.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Service.java
@@ -71,6 +71,9 @@ public class Service extends BaseResource {
   private ServiceState state = null;
   private Map<String, String> quicklinks = new HashMap<>();
   private String queue = null;
+  @JsonProperty("kerberos_principal")
+  @XmlElement(name = "kerberos_principal")
+  private KerberosPrincipal kerberosPrincipal = new KerberosPrincipal();
 
   /**
    * A unique service name.
@@ -335,6 +338,24 @@ public class Service extends BaseResource {
     this.queue = queue;
   }
 
+  public Service kerberosPrincipal(KerberosPrincipal kerberosPrincipal) {
+    this.kerberosPrincipal = kerberosPrincipal;
+    return this;
+  }
+
+  /**
+   * The Kerberos Principal of the service.
+   * @return kerberosPrincipal
+   **/
+  @ApiModelProperty(value = "The Kerberos Principal of the service")
+  public KerberosPrincipal getKerberosPrincipal() {
+    return kerberosPrincipal;
+  }
+
+  public void setKerberosPrincipal(KerberosPrincipal kerberosPrincipal) {
+    this.kerberosPrincipal = kerberosPrincipal;
+  }
+
   @Override
   public boolean equals(java.lang.Object o) {
     if (this == o) {
@@ -376,6 +397,8 @@ public class Service extends BaseResource {
     sb.append("    quicklinks: ").append(toIndentedString(quicklinks))
         .append("\n");
     sb.append("    queue: ").append(toIndentedString(queue)).append("\n");
+    sb.append("    kerberosPrincipal: ")
+        .append(toIndentedString(kerberosPrincipal)).append("\n");
     sb.append("}");
     return sb.toString();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java
index d1b6026..81c56d2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java
@@ -28,12 +28,15 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.registry.client.api.RegistryConstants;
 import org.apache.hadoop.registry.client.api.RegistryOperations;
 import org.apache.hadoop.registry.client.api.RegistryOperationsFactory;
 import org.apache.hadoop.registry.client.binding.RegistryUtils;
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
@@ -43,6 +46,7 @@ import org.apache.hadoop.yarn.api.records.*;
 import org.apache.hadoop.yarn.client.api.AppAdminClient;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.client.api.YarnClientApplication;
+import org.apache.hadoop.yarn.client.util.YarnClientUtils;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
@@ -79,6 +83,9 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.ByteBuffer;
 import java.text.MessageFormat;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
@@ -98,7 +105,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
   //TODO disable retry so that client / rest API doesn't block?
   protected YarnClient yarnClient;
   // Avoid looking up applicationId from fs all the time.
-  private Map<String, ApplicationId> cachedAppIds = new ConcurrentHashMap<>();
+  private Map<String, AppInfo> cachedAppInfo = new ConcurrentHashMap<>();
 
   private RegistryOperations registryClient;
   private CuratorFramework curatorClient;
@@ -210,7 +217,8 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     // Write the definition first and then submit - AM will read the definition
     createDirAndPersistApp(appDir, service);
     ApplicationId appId = submitApp(service);
-    cachedAppIds.put(serviceName, appId);
+    cachedAppInfo.put(serviceName, new AppInfo(appId, service
+        .getKerberosPrincipal().getPrincipalName()));
     service.setId(appId.toString());
     // update app definition with appId
     persistAppDef(appDir, service);
@@ -224,8 +232,9 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     Service persistedService =
         ServiceApiUtil.loadService(fs, serviceName);
     if (!StringUtils.isEmpty(persistedService.getId())) {
-      cachedAppIds.put(persistedService.getName(),
-          ApplicationId.fromString(persistedService.getId()));
+      cachedAppInfo.put(persistedService.getName(), new AppInfo(
+          ApplicationId.fromString(persistedService.getId()),
+          persistedService.getKerberosPrincipal().getPrincipalName()));
     } else {
       throw new YarnException(persistedService.getName()
           + " appId is null, may be not submitted to YARN yet");
@@ -278,8 +287,9 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
       throw new YarnException(
           serviceName + " appId is null, may be not submitted to YARN yet");
     }
-    cachedAppIds.put(persistedService.getName(),
-        ApplicationId.fromString(persistedService.getId()));
+    cachedAppInfo.put(persistedService.getName(), new AppInfo(
+        ApplicationId.fromString(persistedService.getId()), persistedService
+        .getKerberosPrincipal().getPrincipalName()));
     return flexComponents(serviceName, componentCounts, persistedService);
   }
 
@@ -328,7 +338,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
       throw new YarnException(serviceName + " AM hostname is empty");
     }
     ClientAMProtocol proxy =
-        createAMProxy(appReport.getHost(), appReport.getRpcPort());
+        createAMProxy(serviceName, appReport);
     proxy.flexComponents(requestBuilder.build());
     for (Map.Entry<String, Long> entry : original.entrySet()) {
       LOG.info("[COMPONENT {}]: number of containers changed from {} to {}",
@@ -366,8 +376,8 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     LOG.info("Stopping service {}, with appId = {}", serviceName, currentAppId);
     try {
       ClientAMProtocol proxy =
-          createAMProxy(report.getHost(), report.getRpcPort());
-      cachedAppIds.remove(serviceName);
+          createAMProxy(serviceName, report);
+      cachedAppInfo.remove(serviceName);
       if (proxy != null) {
         // try to stop the app gracefully.
         StopRequestProto request = StopRequestProto.newBuilder().build();
@@ -406,8 +416,8 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
         }
       }
     } catch (IOException | YarnException | InterruptedException e) {
-      LOG.info("Failed to stop " + serviceName
-          + " gracefully, forcefully kill the app.");
+      LOG.info("Failed to stop " + serviceName + " gracefully due to: "
+          + e.getMessage() + ", forcefully kill the app.");
       yarnClient.killApplication(currentAppId, "Forcefully kill the app");
     }
     return EXIT_SUCCESS;
@@ -421,7 +431,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     Path appDir = fs.buildClusterDirPath(serviceName);
     FileSystem fileSystem = fs.getFileSystem();
     // remove from the appId cache
-    cachedAppIds.remove(serviceName);
+    cachedAppInfo.remove(serviceName);
     if (fileSystem.exists(appDir)) {
       if (fileSystem.delete(appDir, true)) {
         LOG.info("Successfully deleted service dir for " + serviceName + ": "
@@ -552,7 +562,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     // copy jars to hdfs and add to localResources
     addJarResource(serviceName, localResources);
     // add keytab if in secure env
-    addKeytabResourceIfSecure(fs, localResources, conf, serviceName);
+    addKeytabResourceIfSecure(fs, localResources, app);
     if (LOG.isDebugEnabled()) {
       printLocalResources(localResources);
     }
@@ -581,6 +591,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     amLaunchContext.setCommands(Collections.singletonList(cmdStr));
     amLaunchContext.setEnvironment(env);
     amLaunchContext.setLocalResources(localResources);
+    addHdfsDelegationTokenIfSecure(amLaunchContext);
     submissionContext.setAMContainerSpec(amLaunchContext);
     yarnClient.submitApplication(submissionContext);
     return submissionContext.getApplicationId();
@@ -771,38 +782,75 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     return appJson;
   }
 
+  private void addHdfsDelegationTokenIfSecure(ContainerLaunchContext amContext)
+      throws IOException {
+    if (!UserGroupInformation.isSecurityEnabled()) {
+      return;
+    }
+    Credentials credentials = new Credentials();
+    String tokenRenewer = YarnClientUtils.getRmPrincipal(getConfig());
+    if (StringUtils.isEmpty(tokenRenewer)) {
+      throw new IOException(
+          "Can't get Master Kerberos principal for the RM to use as renewer");
+    }
+    // Get hdfs dt
+    final org.apache.hadoop.security.token.Token<?>[] tokens =
+        fs.getFileSystem().addDelegationTokens(tokenRenewer, credentials);
+    if (tokens != null && tokens.length != 0) {
+      for (Token<?> token : tokens) {
+        LOG.debug("Got DT: " + token);
+      }
+      DataOutputBuffer dob = new DataOutputBuffer();
+      credentials.writeTokenStorageToStream(dob);
+      ByteBuffer fsTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+      amContext.setTokens(fsTokens);
+    }
+  }
+
   private void addKeytabResourceIfSecure(SliderFileSystem fileSystem,
-      Map<String, LocalResource> localResource, Configuration conf,
-      String serviceName) throws IOException, BadConfigException {
+      Map<String, LocalResource> localResource, Service service)
+      throws IOException, YarnException {
     if (!UserGroupInformation.isSecurityEnabled()) {
       return;
     }
-    String keytabPreInstalledOnHost =
-        conf.get(YarnServiceConf.KEY_AM_KEYTAB_LOCAL_PATH);
-    if (StringUtils.isEmpty(keytabPreInstalledOnHost)) {
-      String amKeytabName =
-          conf.get(YarnServiceConf.KEY_AM_LOGIN_KEYTAB_NAME);
-      String keytabDir = conf.get(YarnServiceConf.KEY_HDFS_KEYTAB_DIR);
-      Path keytabPath =
-          fileSystem.buildKeytabPath(keytabDir, amKeytabName, serviceName);
-      if (fileSystem.getFileSystem().exists(keytabPath)) {
-        LocalResource keytabRes =
-            fileSystem.createAmResource(keytabPath, LocalResourceType.FILE);
-        localResource
-            .put(YarnServiceConstants.KEYTAB_DIR + "/" + amKeytabName, keytabRes);
-        LOG.info("Adding AM keytab on hdfs: " + keytabPath);
-      } else {
-        LOG.warn("No keytab file was found at {}.", keytabPath);
-        if (conf.getBoolean(YarnServiceConf.KEY_AM_LOGIN_KEYTAB_REQUIRED, false)) {
-          throw new BadConfigException("No keytab file was found at %s.",
-              keytabPath);
-        } else {
-          LOG.warn("The AM will be "
-              + "started without a kerberos authenticated identity. "
-              + "The service is therefore not guaranteed to remain "
-              + "operational beyond 24 hours.");
-        }
+    String principalName = service.getKerberosPrincipal().getPrincipalName();
+    if (StringUtils.isEmpty(principalName)) {
+      LOG.warn("No Kerberos principal name specified for " + service.getName());
+      return;
+    }
+    if(StringUtils.isEmpty(service.getKerberosPrincipal().getKeytab())) {
+      LOG.warn("No Kerberos keytab specified for " + service.getName());
+      return;
+    }
+
+    URI keytabURI;
+    try {
+      keytabURI = new URI(service.getKerberosPrincipal().getKeytab());
+    } catch (URISyntaxException e) {
+      throw new YarnException(e);
+    }
+
+    switch (keytabURI.getScheme()) {
+    case "hdfs":
+      Path keytabOnhdfs = new Path(keytabURI);
+      if (!fileSystem.getFileSystem().exists(keytabOnhdfs)) {
+        LOG.warn(service.getName() + "'s keytab (principalName = " +
+            principalName + ") doesn't exist at: " + keytabOnhdfs);
+        return;
       }
+      LocalResource keytabRes =
+          fileSystem.createAmResource(keytabOnhdfs, LocalResourceType.FILE);
+      localResource.put(String.format(YarnServiceConstants.KEYTAB_LOCATION,
+          service.getName()), keytabRes);
+      LOG.debug("Adding " + service.getName() + "'s keytab for " +
+          "localization, uri = " + keytabOnhdfs);
+      break;
+    case "file":
+      LOG.debug("Using a keytab from localhost: " + keytabURI);
+      break;
+    default:
+      LOG.warn("Unsupported URI scheme " + keytabURI);
+      break;
     }
   }
 
@@ -856,7 +904,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
       return "";
     }
     ClientAMProtocol amProxy =
-        createAMProxy(appReport.getHost(), appReport.getRpcPort());
+        createAMProxy(appReport.getName(), appReport);
     GetStatusResponseProto response =
         amProxy.getStatus(GetStatusRequestProto.newBuilder().build());
     return response.getStatus();
@@ -886,7 +934,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
       return appSpec;
     }
     ClientAMProtocol amProxy =
-        createAMProxy(appReport.getHost(), appReport.getRpcPort());
+        createAMProxy(serviceName, appReport);
     GetStatusResponseProto response =
         amProxy.getStatus(GetStatusRequestProto.newBuilder().build());
     appSpec = jsonSerDeser.fromJson(response.getStatus());
@@ -935,18 +983,37 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     }
   }
 
-  protected ClientAMProtocol createAMProxy(String host, int port)
-      throws IOException {
+  protected ClientAMProtocol createAMProxy(String serviceName,
+      ApplicationReport appReport) throws IOException, YarnException {
+
+    if (UserGroupInformation.isSecurityEnabled()) {
+      if (!cachedAppInfo.containsKey(serviceName)) {
+        Service persistedService  = ServiceApiUtil.loadService(fs, serviceName);
+        cachedAppInfo.put(serviceName, new AppInfo(appReport.getApplicationId(),
+            persistedService.getKerberosPrincipal().getPrincipalName()));
+      }
+      String principalName = cachedAppInfo.get(serviceName).principalName;
+      // Inject the principal into hadoop conf, because Hadoop
+      // SaslRpcClient#getServerPrincipal requires a config for the
+      // principal
+      if (!StringUtils.isEmpty(principalName)) {
+        getConfig().set(PRINCIPAL, principalName);
+      } else {
+        throw new YarnException("No principal specified in the persisted " +
+            "service definition, fail to connect to AM.");
+      }
+    }
     InetSocketAddress address =
-        NetUtils.createSocketAddrForHost(host, port);
+        NetUtils.createSocketAddrForHost(appReport.getHost(), appReport
+            .getRpcPort());
     return ClientAMProxy.createProxy(getConfig(), ClientAMProtocol.class,
         UserGroupInformation.getCurrentUser(), rpc, address);
   }
 
   public synchronized ApplicationId getAppId(String serviceName)
       throws IOException, YarnException {
-    if (cachedAppIds.containsKey(serviceName)) {
-      return cachedAppIds.get(serviceName);
+    if (cachedAppInfo.containsKey(serviceName)) {
+      return cachedAppInfo.get(serviceName).appId;
     }
     Service persistedService = ServiceApiUtil.loadService(fs, serviceName);
     if (persistedService == null) {
@@ -954,7 +1021,18 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
           + " doesn't exist on hdfs. Please check if the app exists in RM");
     }
     ApplicationId currentAppId = ApplicationId.fromString(persistedService.getId());
-    cachedAppIds.put(serviceName, currentAppId);
+    cachedAppInfo.put(serviceName, new AppInfo(currentAppId, persistedService
+        .getKerberosPrincipal().getPrincipalName()));
     return currentAppId;
   }
+
+  private static class AppInfo {
+    ApplicationId appId;
+    String principalName;
+
+    AppInfo(ApplicationId appId, String principalName) {
+      this.appId = appId;
+      this.principalName = principalName;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
index 88f4763..4e05e5f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
@@ -101,7 +101,7 @@ public class Component implements EventHandler<ComponentEvent> {
       new StateMachineFactory<Component, ComponentState, ComponentEventType, ComponentEvent>(
           INIT)
            // INIT will only got to FLEXING
-          .addTransition(INIT, EnumSet.of(STABLE, FLEXING),
+          .addTransition(INIT, EnumSet.of(STABLE, FLEXING, INIT),
               FLEX, new FlexComponentTransition())
           // container recovered on AM restart
           .addTransition(INIT, INIT, CONTAINER_RECOVERED,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConf.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConf.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConf.java
index 684d980..ea8904a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConf.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConf.java
@@ -53,13 +53,6 @@ public class YarnServiceConf {
    */
   public static final String YARN_SERVICE_BASE_PATH = "yarn.service.base.path";
 
-  //TODO rename
-  /** Declare that a keytab must be provided */
-  public static final String KEY_AM_LOGIN_KEYTAB_REQUIRED = "slider.am.login.keytab.required";
-  public static final String KEY_AM_LOGIN_KEYTAB_NAME = "slider.am.login.keytab.name";
-  public static final String KEY_HDFS_KEYTAB_DIR = "slider.hdfs.keytab.dir";
-  public static final String KEY_AM_KEYTAB_LOCAL_PATH = "slider.am.keytab.local.path";
-
   /**
    * maximum number of failed containers (in a single component)
    * before the app exits

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConstants.java
index 3973759..0378d24 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConstants.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConstants.java
@@ -40,6 +40,8 @@ public interface YarnServiceConstants {
   String APP_TYPE = "yarn-service";
 
   String KEYTAB_DIR = "keytabs";
+  String KEYTAB_LOCATION = KEYTAB_DIR + "/%s" + ".keytab";
+
   String RESOURCE_DIR = "resources";
 
 
@@ -89,4 +91,5 @@ public interface YarnServiceConstants {
   String ERR_FILE = "stderr.txt";
 
   String CONTENT = "content";
+  String PRINCIPAL = "yarn.service.am.principal";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/AbstractLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/AbstractLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/AbstractLauncher.java
index 2d7c3bb..e1e88cd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/AbstractLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/AbstractLauncher.java
@@ -19,16 +19,15 @@
 package org.apache.hadoop.yarn.service.containerlaunch;
 
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerRetryContext;
 import org.apache.hadoop.yarn.api.records.ContainerRetryPolicy;
 import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.util.Records;
+import org.apache.hadoop.yarn.service.ServiceContext;
 import org.apache.hadoop.yarn.service.conf.YarnServiceConstants;
-import org.apache.hadoop.yarn.service.utils.CoreFileSystem;
 import org.apache.hadoop.yarn.service.utils.ServiceUtils;
+import org.apache.hadoop.yarn.util.Records;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -50,10 +49,6 @@ public class AbstractLauncher {
     LoggerFactory.getLogger(AbstractLauncher.class);
   public static final String CLASSPATH = "CLASSPATH";
   /**
-   * Filesystem to use for the launch
-   */
-  protected final CoreFileSystem coreFileSystem;
-  /**
    * Env vars; set up at final launch stage
    */
   protected final Map<String, String> envVars = new HashMap<>();
@@ -63,25 +58,15 @@ public class AbstractLauncher {
   protected final Map<String, LocalResource> localResources = new HashMap<>();
   protected final Map<String, String> mountPaths = new HashMap<>();
   private final Map<String, ByteBuffer> serviceData = new HashMap<>();
-  // security
-  protected final Credentials credentials;
   protected boolean yarnDockerMode = false;
   protected String dockerImage;
   protected String dockerNetwork = DEFAULT_DOCKER_NETWORK;
   protected String dockerHostname;
   protected String runPrivilegedContainer;
+  private ServiceContext context;
 
-
-  /**
-   * Create instance.
-   * @param coreFileSystem filesystem
-   * @param credentials initial set of credentials -null is permitted
-   */
-  public AbstractLauncher(
-      CoreFileSystem coreFileSystem,
-      Credentials credentials) {
-    this.coreFileSystem = coreFileSystem;
-    this.credentials = credentials != null ? credentials: new Credentials();
+  public AbstractLauncher(ServiceContext context) {
+    this.context = context;
   }
   
   public void setYarnDockerMode(boolean yarnDockerMode){
@@ -113,14 +98,6 @@ public class AbstractLauncher {
     mountPaths.put(subPath, mountPath);
   }
 
-  /**
-   * Accessor to the credentials
-   * @return the credentials associated with this launcher
-   */
-  public Credentials getCredentials() {
-    return credentials;
-  }
-
 
   public void addCommand(String cmd) {
     commands.add(cmd);
@@ -160,9 +137,9 @@ public class AbstractLauncher {
     containerLaunchContext.setLocalResources(localResources);
 
     //tokens
-    log.debug("{} tokens", credentials.numberOfTokens());
-    containerLaunchContext.setTokens(CredentialUtils.marshallCredentials(
-        credentials));
+    if (context.tokens != null) {
+      containerLaunchContext.setTokens(context.tokens.duplicate());
+    }
 
     if(yarnDockerMode){
       Map<String, String> env = containerLaunchContext.getEnvironment();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java
index b9f3a24..e07661b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.service.containerlaunch;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.service.ServiceContext;
 import org.apache.hadoop.yarn.service.api.records.Component;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
 import org.apache.hadoop.yarn.service.provider.ProviderService;
@@ -40,10 +41,11 @@ public class ContainerLaunchService extends AbstractService{
 
   private ExecutorService executorService;
   private SliderFileSystem fs;
-
-  public ContainerLaunchService(SliderFileSystem fs) {
+  private ServiceContext context;
+  public ContainerLaunchService(ServiceContext context) {
     super(ContainerLaunchService.class.getName());
-    this.fs = fs;
+    this.fs = context.fs;
+    this.context = context;
   }
 
   @Override
@@ -84,7 +86,7 @@ public class ContainerLaunchService extends AbstractService{
       Component compSpec = instance.getCompSpec();
       ProviderService provider = ProviderFactory.getProviderService(
           compSpec.getArtifact());
-      AbstractLauncher launcher = new AbstractLauncher(fs, null);
+      AbstractLauncher launcher = new AbstractLauncher(context);
       try {
         provider.buildContainerLaunchContext(launcher, service,
             instance, fs, getConfig(), container);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CredentialUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CredentialUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CredentialUtils.java
deleted file mode 100644
index fce58e5..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/CredentialUtils.java
+++ /dev/null
@@ -1,319 +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.service.containerlaunch;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
-import org.apache.hadoop.yarn.client.ClientRMProxy;
-import org.apache.hadoop.yarn.client.api.TimelineClient;
-import org.apache.hadoop.yarn.client.api.YarnClient;
-import org.apache.hadoop.yarn.conf.HAUtil;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
-import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.Serializable;
-import java.nio.ByteBuffer;
-import java.text.DateFormat;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.Iterator;
-import java.util.List;
-
-import static org.apache.hadoop.yarn.conf.YarnConfiguration.*;
-
-/**
- * Utils to work with credentials and tokens.
- *
- * Designed to be movable to Hadoop core
- */
-public final class CredentialUtils {
-
-  private CredentialUtils() {
-  }
-
-  private static final Logger LOG =
-      LoggerFactory.getLogger(CredentialUtils.class);
-
-  /**
-   * Save credentials to a byte buffer. Returns null if there were no
-   * credentials to save
-   * @param credentials credential set
-   * @return a byte buffer of serialized tokens
-   * @throws IOException if the credentials could not be written to the stream
-   */
-  public static ByteBuffer marshallCredentials(Credentials credentials) throws IOException {
-    ByteBuffer buffer = null;
-    if (!credentials.getAllTokens().isEmpty()) {
-      DataOutputBuffer dob = new DataOutputBuffer();
-      try {
-        credentials.writeTokenStorageToStream(dob);
-      } finally {
-        dob.close();
-      }
-      buffer = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
-    }
-    return buffer;
-  }
-
-  /**
-   * Save credentials to a file
-   * @param file file to save to (will be overwritten)
-   * @param credentials credentials to write
-   * @throws IOException
-   */
-  public static void saveTokens(File file,
-      Credentials credentials) throws IOException {
-    try(DataOutputStream daos = new DataOutputStream(
-        new FileOutputStream(file))) {
-      credentials.writeTokenStorageToStream(daos);
-    }
-  }
-
-  /**
-   * Look up and return the resource manager's principal. This method
-   * automatically does the <code>_HOST</code> replacement in the principal and
-   * correctly handles HA resource manager configurations.
-   *
-   * From: YARN-4629
-   * @param conf the {@link Configuration} file from which to read the
-   * principal
-   * @return the resource manager's principal string
-   * @throws IOException thrown if there's an error replacing the host name
-   */
-  public static String getRMPrincipal(Configuration conf) throws IOException {
-    String principal = conf.get(RM_PRINCIPAL, "");
-    String hostname;
-    Preconditions.checkState(!principal.isEmpty(), "Not set: " + RM_PRINCIPAL);
-
-    if (HAUtil.isHAEnabled(conf)) {
-      YarnConfiguration yarnConf = new YarnConfiguration(conf);
-      if (yarnConf.get(RM_HA_ID) == null) {
-        // If RM_HA_ID is not configured, use the first of RM_HA_IDS.
-        // Any valid RM HA ID should work.
-        String[] rmIds = yarnConf.getStrings(RM_HA_IDS);
-        Preconditions.checkState((rmIds != null) && (rmIds.length > 0),
-            "Not set " + RM_HA_IDS);
-        yarnConf.set(RM_HA_ID, rmIds[0]);
-      }
-
-      hostname = yarnConf.getSocketAddr(
-          RM_ADDRESS,
-          DEFAULT_RM_ADDRESS,
-          DEFAULT_RM_PORT).getHostName();
-    } else {
-      hostname = conf.getSocketAddr(
-          RM_ADDRESS,
-          DEFAULT_RM_ADDRESS,
-          DEFAULT_RM_PORT).getHostName();
-    }
-    return SecurityUtil.getServerPrincipal(principal, hostname);
-  }
-
-  /**
-   * Create and add any filesystem delegation tokens with
-   * the RM(s) configured to be able to renew them. Returns null
-   * on an insecure cluster (i.e. harmless)
-   * @param conf configuration
-   * @param fs filesystem
-   * @param credentials credentials to update
-   * @return a list of all added tokens.
-   * @throws IOException
-   */
-  public static Token<?>[] addRMRenewableFSDelegationTokens(Configuration conf,
-      FileSystem fs,
-      Credentials credentials) throws IOException {
-    Preconditions.checkArgument(conf != null);
-    Preconditions.checkArgument(credentials != null);
-    if (UserGroupInformation.isSecurityEnabled()) {
-      return fs.addDelegationTokens(CredentialUtils.getRMPrincipal(conf),
-          credentials);
-    }
-    return null;
-  }
-
-  /**
-   * Add an FS delegation token which can be renewed by the current user
-   * @param fs filesystem
-   * @param credentials credentials to update
-   * @throws IOException problems.
-   */
-  public static void addSelfRenewableFSDelegationTokens(
-      FileSystem fs,
-      Credentials credentials) throws IOException {
-    Preconditions.checkArgument(fs != null);
-    Preconditions.checkArgument(credentials != null);
-    fs.addDelegationTokens(
-        getSelfRenewer(),
-        credentials);
-  }
-
-  public static String getSelfRenewer() throws IOException {
-    return UserGroupInformation.getLoginUser().getShortUserName();
-  }
-
-  /**
-   * Create and add an RM delegation token to the credentials
-   * @param yarnClient Yarn Client
-   * @param credentials to add token to
-   * @return the token which was added
-   * @throws IOException
-   * @throws YarnException
-   */
-  public static Token<TokenIdentifier> addRMDelegationToken(YarnClient yarnClient,
-      Credentials credentials)
-      throws IOException, YarnException {
-    Configuration conf = yarnClient.getConfig();
-    Text rmPrincipal = new Text(CredentialUtils.getRMPrincipal(conf));
-    Text rmDTService = ClientRMProxy.getRMDelegationTokenService(conf);
-    Token<TokenIdentifier> rmDelegationToken =
-        ConverterUtils.convertFromYarn(
-            yarnClient.getRMDelegationToken(rmPrincipal),
-            rmDTService);
-    credentials.addToken(rmDelegationToken.getService(), rmDelegationToken);
-    return rmDelegationToken;
-  }
-
-  public static Token<TimelineDelegationTokenIdentifier> maybeAddTimelineToken(
-      Configuration conf,
-      Credentials credentials)
-      throws IOException, YarnException {
-    if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, false)) {
-      LOG.debug("Timeline service enabled -fetching token");
-
-      try(TimelineClient timelineClient = TimelineClient.createTimelineClient()) {
-        timelineClient.init(conf);
-        timelineClient.start();
-        Token<TimelineDelegationTokenIdentifier> token =
-            timelineClient.getDelegationToken(
-                CredentialUtils.getRMPrincipal(conf));
-        credentials.addToken(token.getService(), token);
-        return token;
-      }
-    } else {
-      LOG.debug("Timeline service is disabled");
-      return null;
-    }
-  }
-
-  /**
-   * Filter a list of tokens from a set of credentials
-   * @param credentials credential source (a new credential set os re
-   * @param filter List of tokens to strip out
-   * @return a new, filtered, set of credentials
-   */
-  public static Credentials filterTokens(Credentials credentials,
-      List<Text> filter) {
-    Credentials result = new Credentials(credentials);
-    Iterator<Token<? extends TokenIdentifier>> iter =
-        result.getAllTokens().iterator();
-    while (iter.hasNext()) {
-      Token<? extends TokenIdentifier> token = iter.next();
-      LOG.debug("Token {}", token.getKind());
-      if (filter.contains(token.getKind())) {
-        LOG.debug("Filtering token {}", token.getKind());
-        iter.remove();
-      }
-    }
-    return result;
-  }
-
-  public static String dumpTokens(Credentials credentials, String separator) {
-    ArrayList<Token<? extends TokenIdentifier>> sorted =
-        new ArrayList<>(credentials.getAllTokens());
-    Collections.sort(sorted, new TokenComparator());
-    StringBuilder buffer = new StringBuilder(sorted.size()* 128);
-    for (Token<? extends TokenIdentifier> token : sorted) {
-      buffer.append(tokenToString(token)).append(separator);
-    }
-    return buffer.toString();
-  }
-
-  /**
-   * Create a string for people to look at
-   * @param token token to convert to a string form
-   * @return a printable view of the token
-   */
-  public static String tokenToString(Token<? extends TokenIdentifier> token) {
-    DateFormat df = DateFormat.getDateTimeInstance(
-        DateFormat.SHORT, DateFormat.SHORT);
-    StringBuilder buffer = new StringBuilder(128);
-    buffer.append(token.toString());
-    try {
-      TokenIdentifier ti = token.decodeIdentifier();
-      buffer.append("; ").append(ti);
-      if (ti instanceof AbstractDelegationTokenIdentifier) {
-        // details in human readable form, and compensate for information HDFS DT omits
-        AbstractDelegationTokenIdentifier dt = (AbstractDelegationTokenIdentifier) ti;
-        buffer.append("; Renewer: ").append(dt.getRenewer());
-        buffer.append("; Issued: ")
-            .append(df.format(new Date(dt.getIssueDate())));
-        buffer.append("; Max Date: ")
-            .append(df.format(new Date(dt.getMaxDate())));
-      }
-    } catch (IOException e) {
-      //marshall problem; not ours
-      LOG.debug("Failed to decode {}: {}", token, e, e);
-    }
-    return buffer.toString();
-  }
-
-  /**
-   * Get the expiry time of a token.
-   * @param token token to examine
-   * @return the time in milliseconds after which the token is invalid.
-   * @throws IOException
-   */
-  public static long getTokenExpiryTime(Token token) throws IOException {
-    TokenIdentifier identifier = token.decodeIdentifier();
-    Preconditions.checkState(identifier instanceof AbstractDelegationTokenIdentifier,
-        "Token %s of type: %s has an identifier which cannot be examined: %s",
-        token, token.getClass(), identifier);
-    AbstractDelegationTokenIdentifier id =
-        (AbstractDelegationTokenIdentifier) identifier;
-    return id.getMaxDate();
-  }
-
-  private static class TokenComparator
-      implements Comparator<Token<? extends TokenIdentifier>>, Serializable {
-    @Override
-    public int compare(Token<? extends TokenIdentifier> left,
-        Token<? extends TokenIdentifier> right) {
-      return left.getKind().toString().compareTo(right.getKind().toString());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/package-info.java
new file mode 100644
index 0000000..766da0d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Yarn Service framework.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.yarn.service;
+import org.apache.hadoop.classification.InterfaceAudience;
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java
index c0c44c3..d65a196 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.yarn.service.provider;
 
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
@@ -28,21 +27,18 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.service.ServiceContext;
-import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.api.records.Component;
 import org.apache.hadoop.yarn.service.api.records.ConfigFile;
 import org.apache.hadoop.yarn.service.api.records.ConfigFormat;
-import org.apache.hadoop.yarn.service.api.records.Configuration;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
 import org.apache.hadoop.yarn.service.conf.YarnServiceConstants;
-import org.apache.hadoop.yarn.service.conf.YarnServiceConf;
 import org.apache.hadoop.yarn.service.containerlaunch.AbstractLauncher;
 import org.apache.hadoop.yarn.service.exceptions.BadCommandArgumentsException;
 import org.apache.hadoop.yarn.service.exceptions.SliderException;
 import org.apache.hadoop.yarn.service.utils.PublishedConfiguration;
 import org.apache.hadoop.yarn.service.utils.PublishedConfigurationOutputter;
-import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
 import org.apache.hadoop.yarn.service.utils.ServiceUtils;
+import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -163,53 +159,6 @@ public class ProviderUtils implements YarnServiceConstants {
     }
   }
 
-  /**
-   * Localize the service keytabs for the service.
-   * @param launcher container launcher
-   * @param fileSystem file system
-   * @throws IOException trouble uploading to HDFS
-   */
-  public void localizeServiceKeytabs(AbstractLauncher launcher,
-      SliderFileSystem fileSystem, Service service) throws IOException {
-
-    Configuration conf = service.getConfiguration();
-    String keytabPathOnHost =
-        conf.getProperty(YarnServiceConf.KEY_AM_KEYTAB_LOCAL_PATH);
-    if (ServiceUtils.isUnset(keytabPathOnHost)) {
-      String amKeytabName =
-          conf.getProperty(YarnServiceConf.KEY_AM_LOGIN_KEYTAB_NAME);
-      String keytabDir =
-          conf.getProperty(YarnServiceConf.KEY_HDFS_KEYTAB_DIR);
-      // we need to localize the keytab files in the directory
-      Path keytabDirPath = fileSystem.buildKeytabPath(keytabDir, null,
-          service.getName());
-      boolean serviceKeytabsDeployed = false;
-      if (fileSystem.getFileSystem().exists(keytabDirPath)) {
-        FileStatus[] keytabs = fileSystem.getFileSystem().listStatus(
-            keytabDirPath);
-        LocalResource keytabRes;
-        for (FileStatus keytab : keytabs) {
-          if (!amKeytabName.equals(keytab.getPath().getName())
-              && keytab.getPath().getName().endsWith(".keytab")) {
-            serviceKeytabsDeployed = true;
-            log.info("Localizing keytab {}", keytab.getPath().getName());
-            keytabRes = fileSystem.createAmResource(keytab.getPath(),
-                LocalResourceType.FILE);
-            launcher.addLocalResource(KEYTAB_DIR + "/" +
-                    keytab.getPath().getName(),
-                keytabRes);
-          }
-        }
-      }
-      if (!serviceKeytabsDeployed) {
-        log.warn("No service keytabs for the service have been localized.  "
-            + "If the service requires keytabs for secure operation, "
-            + "please ensure that the required keytabs have been uploaded "
-            + "to the folder {}", keytabDirPath);
-      }
-    }
-  }
-
   public static Path initCompInstanceDir(SliderFileSystem fs,
       ComponentInstance instance) {
     Path compDir = new Path(new Path(fs.getAppDir(), "components"),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
index b58cea8..d5ea45c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
@@ -24,6 +24,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.registry.client.api.RegistryConstants;
 import org.apache.hadoop.registry.client.binding.RegistryUtils;
+import org.apache.hadoop.security.HadoopKerberosName;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.api.records.Artifact;
@@ -40,6 +42,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
@@ -86,6 +90,17 @@ public class ServiceApiUtil {
           "No component specified for " + service.getName());
     }
 
+    if (UserGroupInformation.isSecurityEnabled()) {
+      if (!StringUtils.isEmpty(service.getKerberosPrincipal().getKeytab())) {
+        try {
+          // validate URI format
+          new URI(service.getKerberosPrincipal().getKeytab());
+        } catch (URISyntaxException e) {
+          throw new IllegalArgumentException(e);
+        }
+      }
+    }
+
     // Validate there are no component name collisions (collisions are not
     // currently supported) and add any components from external services
     Configuration globalConf = service.getConfiguration();


---------------------------------------------------------------------
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-15071 S3a troubleshooting docs to add a couple more failure modes. Contributed by Steve Loughran

Posted by ae...@apache.org.
HADOOP-15071 S3a troubleshooting docs to add a couple more failure modes.
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/3150c019
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3150c019
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3150c019

Branch: refs/heads/HDFS-7240
Commit: 3150c019aef21edf132a4f72260417f36036d89a
Parents: 6d16a99
Author: Steve Loughran <st...@apache.org>
Authored: Tue Dec 5 15:05:41 2017 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Dec 5 15:05:41 2017 +0000

----------------------------------------------------------------------
 .../tools/hadoop-aws/troubleshooting_s3a.md     | 140 ++++++++++++++++++-
 1 file changed, 139 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3150c019/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
index 7d16744..5caadd7 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
@@ -228,6 +228,41 @@ As an example, the endpoint for S3 Frankfurt is `s3.eu-central-1.amazonaws.com`:
 </property>
 ```
 
+
+### "403 Access denied" when trying to write data
+
+Data can be read, but attempts to write data or manipulate the store fail with
+403/Access denied.
+
+The bucket may have an access policy which the request does not comply with.
+
+```
+java.nio.file.AccessDeniedException: test/: PUT 0-byte object  on test/:
+ com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied (Service: Amazon S3; Status Code: 403;
+ Error Code: AccessDenied; Request ID: EDC662AD2EEEA33C;
+  at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:210)
+  at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:110)
+  at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:259)
+  at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:313)
+  at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:255)
+  at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:230)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.createEmptyObject(S3AFileSystem.java:2691)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.createFakeDirectory(S3AFileSystem.java:2666)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.innerMkdirs(S3AFileSystem.java:2030)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1965)
+  at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2305)
+```
+
+In the AWS S3 management console, select the "permissions" tab for the bucket, then "bucket policy".
+If there is no bucket policy, then the error cannot be caused by one.
+
+If there is a bucket access policy, e.g. required encryption headers,
+then the settings of the s3a client must guarantee the relevant headers are set
+(e.g. the encryption options match).
+Note: S3 Default Encryption options are not considered here:
+if the bucket policy requires AES256 as the encryption policy on PUT requests,
+then the encryption option must be set in the s3a client so that the header is set.
+
 ## <a name="connectivity"></a> Connectivity Problems
 
 ### <a name="bad_endpoint"></a> Error message "The bucket you are attempting to access must be addressed using the specified endpoint"
@@ -412,8 +447,13 @@ It may go away if the operation is retried.
 ### When writing data: "java.io.FileNotFoundException: Completing multi-part upload"
 
 
+A multipart upload was trying to complete, but failed as there was no upload
+with that ID.
 ```
-java.io.FileNotFoundException: Completing multi-part upload on fork-5/test/multipart/1c397ca6-9dfb-4ac1-9cf7-db666673246b: com.amazonaws.services.s3.model.AmazonS3Exception: The specified upload does not exist. The upload ID may be invalid, or the upload may have been aborted or completed. (Service: Amazon S3; Status Code: 404; Error Code: NoSuchUpload; Request ID: 84FF8057174D9369), S3 Extended Request ID: Ij5Yn6Eq/qIERH4Z6Io3YL2t9/qNZ7z9gjPb1FrTtTovZ8k1MXqh+zCYYjqmfJ/fCY6E1+JR9jA=
+java.io.FileNotFoundException: Completing multi-part upload on fork-5/test/multipart/1c397ca6-9dfb-4ac1-9cf7-db666673246b:
+ com.amazonaws.services.s3.model.AmazonS3Exception: The specified upload does not exist.
+  The upload ID may be invalid, or the upload may have been aborted or completed. (Service: Amazon S3; Status Code: 404;
+   Error Code: NoSuchUpload;
   at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182)
   at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770)
   at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
@@ -427,6 +467,104 @@ java.io.FileNotFoundException: Completing multi-part upload on fork-5/test/multi
   at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
 ```
 
+This can happen when all outstanding uploads have been aborted, including
+the active ones.
+
+If the bucket has a lifecycle policy of deleting multipart uploads, make
+sure that the expiry time of the deletion is greater than that required
+for all open writes to complete the write,
+*and for all jobs using the S3A committers to commit their work.*
+
+
+### Application hangs after reading a number of files
+
+
+
+
+The pool of https client connectons and/or IO threads have been used up,
+and none are being freed.
+
+
+1. The pools aren't big enough. Increas `fs.s3a.connection.maximum` for
+the http connections, and `fs.s3a.threads.max` for the thread pool.
+2. Likely root cause: whatever code is reading files isn't calling `close()`
+on the input streams. Make sure your code does this!
+And if it's someone else's: make sure you have a recent version; search their
+issue trackers to see if its a known/fixed problem.
+If not, it's time to work with the developers, or come up with a workaround
+(i.e closing the input stream yourself).
+
+### "Timeout waiting for connection from pool"
+
+This the same problem as above, exhibiting itself as the http connection
+pool determining that it has run out of capacity.
+
+```
+
+java.io.InterruptedIOException: getFileStatus on s3a://example/fork-0007/test:
+ com.amazonaws.SdkClientException: Unable to execute HTTP request: Timeout waiting for connection from pool
+  at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:145)
+  at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:119)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:2040)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.checkPathForDirectory(S3AFileSystem.java:1857)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.innerMkdirs(S3AFileSystem.java:1890)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1826)
+  at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2230)
+  ...
+Caused by: com.amazonaws.SdkClientException: Unable to execute HTTP request: Timeout waiting for connection from pool
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleRetryableException(AmazonHttpClient.java:1069)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1035)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:742)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:716)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
+  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
+  at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4221)
+  at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4168)
+  at com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1249)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:1162)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:2022)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.checkPathForDirectory(S3AFileSystem.java:1857)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.innerMkdirs(S3AFileSystem.java:1890)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1826)
+  at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2230)
+...
+Caused by: com.amazonaws.thirdparty.apache.http.conn.ConnectionPoolTimeoutException: Timeout waiting for connection from pool
+  at com.amazonaws.thirdparty.apache.http.impl.conn.PoolingHttpClientConnectionManager.leaseConnection(PoolingHttpClientConnectionManager.java:286)
+  at com.amazonaws.thirdparty.apache.http.impl.conn.PoolingHttpClientConnectionManager$1.get(PoolingHttpClientConnectionManager.java:263)
+  at sun.reflect.GeneratedMethodAccessor10.invoke(Unknown Source)
+  at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
+  at java.lang.reflect.Method.invoke(Method.java:498)
+  at com.amazonaws.http.conn.ClientConnectionRequestFactory$Handler.invoke(ClientConnectionRequestFactory.java:70)
+  at com.amazonaws.http.conn.$Proxy15.get(Unknown Source)
+  at com.amazonaws.thirdparty.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:190)
+  at com.amazonaws.thirdparty.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:184)
+  at com.amazonaws.thirdparty.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:184)
+  at com.amazonaws.thirdparty.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
+  at com.amazonaws.thirdparty.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
+  at com.amazonaws.http.apache.client.impl.SdkHttpClient.execute(SdkHttpClient.java:72)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1190)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1030)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:742)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:716)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
+  at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
+  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
+  at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4221)
+  at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4168)
+  at com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1249)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:1162)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:2022)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.checkPathForDirectory(S3AFileSystem.java:1857)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.innerMkdirs(S3AFileSystem.java:1890)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1826)
+  at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2230)
+```
+
+This is the same problem as the previous one, exhibited differently.
+
 ### Issue: when writing data, HTTP Exceptions logged at info from `AmazonHttpClient`
 
 ```


---------------------------------------------------------------------
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: HADOOP-13493. Compatibility Docs should clarify the policy for what takes precedence when a conflict is found (templedf via rkanter)

Posted by ae...@apache.org.
HADOOP-13493. Compatibility Docs should clarify the policy for what takes precedence when a conflict is found (templedf via rkanter)


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

Branch: refs/heads/HDFS-7240
Commit: 75a3ab88f5f4ea6abf0a56cb8058e17b5a5fe403
Parents: 0e560f3
Author: Robert Kanter <rk...@apache.org>
Authored: Thu Nov 30 07:39:15 2017 -0800
Committer: Robert Kanter <rk...@apache.org>
Committed: Thu Nov 30 07:39:15 2017 -0800

----------------------------------------------------------------------
 .../src/site/markdown/Compatibility.md          | 29 +++++++++++++++-----
 1 file changed, 22 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/75a3ab88/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md b/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
index 461ff17..54be412 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
@@ -117,13 +117,7 @@ Compatibility types
 
 Developers SHOULD annotate all Hadoop interfaces and classes with the
 @InterfaceAudience and @InterfaceStability annotations to describe the
-intended audience and stability. Annotations may be at the package, class, or
-member variable or method level. Member variable and method annotations SHALL
-override class annotations, and class annotations SHALL override package
-annotations. A package, class, or member variable or method that is not
-annotated SHALL be interpreted as implicitly
-[Private](./InterfaceClassification.html#Private) and
-[Unstable](./InterfaceClassification.html#Unstable).
+intended audience and stability.
 
 * @InterfaceAudience captures the intended audience. Possible values are
 [Public](./InterfaceClassification.html#Public) (for end users and external
@@ -134,6 +128,27 @@ etc.), and [Private](./InterfaceClassification.html#Private)
 * @InterfaceStability describes what types of interface changes are permitted. Possible values are [Stable](./InterfaceClassification.html#Stable), [Evolving](./InterfaceClassification.html#Evolving), and [Unstable](./InterfaceClassification.html#Unstable).
 * @Deprecated notes that the package, class, or member variable or method could potentially be removed in the future and should not be used.
 
+Annotations MAY be applied at the package, class, or method level. If a method
+has no privacy or stability annotation, it SHALL inherit its intended audience
+or stability level from the class to which it belongs. If a class has no
+privacy or stability annotation, it SHALL inherit its intended audience or
+stability level from the package to which it belongs. If a package has no
+privacy or stability annotation, it SHALL be assumed to be
+[Private](./InterfaceClassification.html#Private) and
+[Unstable](./InterfaceClassification.html#Unstable),
+respectively.
+
+In the event that an element's audience or stability annotation conflicts with
+the corresponding annotation of its parent (whether explicit or inherited), the
+element's audience or stability (respectively) SHALL be determined by the
+more restrictive annotation. For example, if a
+[Private](./InterfaceClassification.html#Private) method is contained
+in a [Public](./InterfaceClassification.html#Public) class, then the method
+SHALL be treated as [Private](./InterfaceClassification.html#Private). If a
+[Public](./InterfaceClassification.html#Public) method is contained in a
+[Private](./InterfaceClassification.html#Private) class, the method SHALL be
+treated as [Private](./InterfaceClassification.html#Private).
+
 #### Use Cases
 
 * [Public](./InterfaceClassification.html#Public)-[Stable](./InterfaceClassification.html#Stable) API compatibility is required to ensure end-user programs and downstream projects continue to work without modification.


---------------------------------------------------------------------
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: HDFS-12594. snapshotDiff fails if the report exceeds the RPC response limit. Contributed by Shashikant Banerjee

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b1c7654e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java
index e0a7b5b..a4fb8ab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java
@@ -90,6 +90,7 @@ public class TestSnapshotDiffReport {
     conf.setBoolean(
         DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT,
         true);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT, 3);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
         .format(true).build();
     cluster.waitActive();
@@ -1293,4 +1294,119 @@ public class TestSnapshotDiffReport {
 
     assertAtimeNotEquals(filePostSS, root, "s2", "s3");
   }
+
+  /**
+   * Tests to verfy the diff report with maximum SnapsdiffReportEntries limit
+   * over an rpc being set to 3.
+   * @throws Exception
+   */
+  @Test
+  public void testDiffReportWithRpcLimit() throws Exception {
+    final Path root = new Path("/");
+    hdfs.mkdirs(root);
+    for (int i = 1; i < 4; i++) {
+      final Path path = new Path(root, "dir" + i);
+      hdfs.mkdirs(path);
+    }
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s0");
+    for (int i = 1; i < 4; i++) {
+      final Path path = new Path(root, "dir" + i);
+      for (int j = 1; j < 4; j++) {
+        final Path file = new Path(path, "file" + j);
+        DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, SEED);
+      }
+    }
+
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s1");
+    verifyDiffReport(root, "s0", "s1",
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir1/file1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir1/file2")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir1/file3")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir2")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir2/file1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir2/file2")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir2/file3")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir3")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir3/file1")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir3/file2")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir3/file3")));
+  }
+
+  @Test
+  public void testDiffReportWithRpcLimit2() throws Exception {
+    final Path root = new Path("/");
+    hdfs.mkdirs(root);
+    for (int i = 1; i <=3; i++) {
+      final Path path = new Path(root, "dir" + i);
+      hdfs.mkdirs(path);
+    }
+    for (int i = 1; i <= 3; i++) {
+      final Path path = new Path(root, "dir" + i);
+      for (int j = 1; j < 4; j++) {
+        final Path file = new Path(path, "file" + j);
+        DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, SEED);
+      }
+    }
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s0");
+    Path targetDir = new Path(root, "dir4");
+    //create directory dir4
+    hdfs.mkdirs(targetDir);
+    //moves files from dir1 to dir4
+    Path path = new Path(root, "dir1");
+    for (int j = 1; j < 4; j++) {
+      final Path srcPath = new Path(path, "file" + j);
+      final Path targetPath = new Path(targetDir, "file" + j);
+      hdfs.rename(srcPath, targetPath);
+    }
+    targetDir = new Path(root, "dir3");
+    //overwrite existing files in dir3 from files in dir1
+    path = new Path(root, "dir2");
+    for (int j = 1; j < 4; j++) {
+      final Path srcPath = new Path(path, "file" + j);
+      final Path targetPath = new Path(targetDir, "file" + j);
+      hdfs.rename(srcPath, targetPath, Rename.OVERWRITE);
+    }
+    final Path pathToRename = new Path(root, "dir2");
+    //move dir2 inside dir3
+    hdfs.rename(pathToRename, targetDir);
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s1");
+    verifyDiffReport(root, "s0", "s1",
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("")),
+        new DiffReportEntry(DiffType.CREATE,
+            DFSUtil.string2Bytes("dir4")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("dir2"),
+            DFSUtil.string2Bytes("dir3/dir2")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir1")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("dir1/file1"),
+            DFSUtil.string2Bytes("dir4/file1")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("dir1/file2"),
+            DFSUtil.string2Bytes("dir4/file2")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("dir1/file3"),
+            DFSUtil.string2Bytes("dir4/file3")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir2")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("dir2/file1"),
+            DFSUtil.string2Bytes("dir3/file1")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("dir2/file2"),
+            DFSUtil.string2Bytes("dir3/file2")),
+        new DiffReportEntry(DiffType.RENAME, DFSUtil.string2Bytes("dir2/file3"),
+            DFSUtil.string2Bytes("dir3/file3")),
+        new DiffReportEntry(DiffType.MODIFY, DFSUtil.string2Bytes("dir3")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("dir3/file1")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("dir3/file1")),
+        new DiffReportEntry(DiffType.DELETE,
+            DFSUtil.string2Bytes("dir3/file3")));
+  }
 }


---------------------------------------------------------------------
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: YARN-7610. Extend Distributed Shell to support launching job with opportunistic containers. Contributed by Weiwei Yang.

Posted by ae...@apache.org.
YARN-7610. Extend Distributed Shell to support launching job with opportunistic containers. 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/40b0045e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/40b0045e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/40b0045e

Branch: refs/heads/HDFS-7240
Commit: 40b0045ebe0752cd3d1d09be00acbabdea983799
Parents: 56b1ff8
Author: Weiwei Yang <ww...@apache.org>
Authored: Wed Dec 6 17:52:41 2017 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Wed Dec 6 17:52:41 2017 +0800

----------------------------------------------------------------------
 .../distributedshell/ApplicationMaster.java     |  22 +-
 .../applications/distributedshell/Client.java   |  19 ++
 .../distributedshell/TestDistributedShell.java  |  56 +++++
 .../site/markdown/OpportunisticContainers.md    | 212 -----------------
 .../site/markdown/OpportunisticContainers.md.vm | 233 +++++++++++++++++++
 5 files changed, 329 insertions(+), 213 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b0045e/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 91dbc00..926de50 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
@@ -41,6 +41,7 @@ import java.util.Vector;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.Arrays;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
@@ -90,6 +91,8 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.UpdatedContainer;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
@@ -241,6 +244,9 @@ public class ApplicationMaster {
   private int containerVirtualCores = DEFAULT_CONTAINER_VCORES;
   // Priority of the request
   private int requestPriority;
+  // Execution type of the containers.
+  // Default GUARANTEED.
+  private ExecutionType containerType = ExecutionType.GUARANTEED;
 
   // Resource profile for the container
   private String containerResourceProfile = "";
@@ -412,6 +418,8 @@ public class ApplicationMaster {
         "App Attempt ID. Not to be used unless for testing purposes");
     opts.addOption("shell_env", true,
         "Environment for shell script. Specified as env_key=env_val pairs");
+    opts.addOption("container_type", true,
+        "Container execution type, GUARANTEED or OPPORTUNISTIC");
     opts.addOption("container_memory", true,
         "Amount of memory in MB to be requested to run the shell command");
     opts.addOption("container_vcores", true,
@@ -558,6 +566,16 @@ public class ApplicationMaster {
       domainId = envs.get(DSConstants.DISTRIBUTEDSHELLTIMELINEDOMAIN);
     }
 
+    if (cliParser.hasOption("container_type")) {
+      String containerTypeStr = cliParser.getOptionValue("container_type");
+      if (Arrays.stream(ExecutionType.values()).noneMatch(
+          executionType -> executionType.toString()
+              .equals(containerTypeStr))) {
+        throw new IllegalArgumentException("Invalid container_type: "
+            + containerTypeStr);
+      }
+      containerType = ExecutionType.valueOf(containerTypeStr);
+    }
     containerMemory = Integer.parseInt(cliParser.getOptionValue(
         "container_memory", "-1"));
     containerVirtualCores = Integer.parseInt(cliParser.getOptionValue(
@@ -1242,7 +1260,9 @@ public class ApplicationMaster {
 
     // Set up resource type requirements
     ContainerRequest request =
-        new ContainerRequest(createProfileCapability(), null, null, pri);
+        new ContainerRequest(createProfileCapability(), null, null,
+            pri, 0, true, null,
+            ExecutionTypeRequest.newInstance(containerType));
     LOG.info("Requested container ask: " + request.toString());
     return request;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b0045e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.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/Client.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
index 0582afe..16bf0fd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
@@ -27,6 +27,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.Vector;
+import java.util.Arrays;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
@@ -74,6 +75,7 @@ import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.client.api.YarnClientApplication;
@@ -171,6 +173,8 @@ public class Client {
   // No. of containers in which the shell script needs to be executed
   private int numContainers = 1;
   private String nodeLabelExpression = null;
+  // Container type, default GUARANTEED.
+  private ExecutionType containerType = ExecutionType.GUARANTEED;
 
   // log4j.properties file 
   // if available, add to local resources and set into classpath 
@@ -282,6 +286,8 @@ public class Client {
     opts.addOption("shell_env", true,
         "Environment for shell script. Specified as env_key=env_val pairs");
     opts.addOption("shell_cmd_priority", true, "Priority for the shell command containers");
+    opts.addOption("container_type", true,
+        "Container execution type, GUARANTEED or OPPORTUNISTIC");
     opts.addOption("container_memory", true, "Amount of memory in MB to be requested to run the shell command");
     opts.addOption("container_vcores", true, "Amount of virtual cores to be requested to run the shell command");
     opts.addOption("container_resource_profile", true, "Resource profile for the shell command");
@@ -433,6 +439,16 @@ public class Client {
     }
     shellCmdPriority = Integer.parseInt(cliParser.getOptionValue("shell_cmd_priority", "0"));
 
+    if (cliParser.hasOption("container_type")) {
+      String containerTypeStr = cliParser.getOptionValue("container_type");
+      if (Arrays.stream(ExecutionType.values()).noneMatch(
+          executionType -> executionType.toString()
+          .equals(containerTypeStr))) {
+        throw new IllegalArgumentException("Invalid container_type: "
+            + containerTypeStr);
+      }
+      containerType = ExecutionType.valueOf(containerTypeStr);
+    }
     containerMemory =
         Integer.parseInt(cliParser.getOptionValue("container_memory", "-1"));
     containerVirtualCores =
@@ -740,6 +756,9 @@ public class Client {
     // Set class name 
     vargs.add(appMasterMainClass);
     // Set params for Application Master
+    if (containerType != null) {
+      vargs.add("--container_type " + String.valueOf(containerType));
+    }
     if (containerMemory > 0) {
       vargs.add("--container_memory " + String.valueOf(containerMemory));
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b0045e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
index 5cf884b..d6bb8d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
@@ -1179,6 +1179,33 @@ public class TestDistributedShell {
           e.getMessage().contains("No shell command or shell script specified " +
           "to be executed by application master"));
     }
+
+    LOG.info("Initializing DS Client with invalid container_type argument");
+    try {
+      String[] args = {
+          "--jar",
+          APPMASTER_JAR,
+          "--num_containers",
+          "2",
+          "--master_memory",
+          "512",
+          "--master_vcores",
+          "2",
+          "--container_memory",
+          "128",
+          "--container_vcores",
+          "1",
+          "--shell_command",
+          "date",
+          "--container_type",
+          "UNSUPPORTED_TYPE"
+      };
+      client.init(args);
+      Assert.fail("Exception is expected");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue("The throw exception is not expected",
+          e.getMessage().contains("Invalid container_type: UNSUPPORTED_TYPE"));
+    }
   }
 
   @Test
@@ -1377,4 +1404,33 @@ public class TestDistributedShell {
       }
     }
   }
+
+  @Test
+  public void testDSShellWithOpportunisticContainers() throws Exception {
+    Client client = new Client(new Configuration(yarnCluster.getConfig()));
+    try {
+      String[] args = {
+          "--jar",
+          APPMASTER_JAR,
+          "--num_containers",
+          "2",
+          "--master_memory",
+          "512",
+          "--master_vcores",
+          "2",
+          "--container_memory",
+          "128",
+          "--container_vcores",
+          "1",
+          "--shell_command",
+          "date",
+          "--container_type",
+          "OPPORTUNISTIC"
+      };
+      client.init(args);
+      client.run();
+    } catch (Exception e) {
+      Assert.fail("Job execution with opportunistic containers failed.");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b0045e/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
deleted file mode 100644
index 83beb07..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md
+++ /dev/null
@@ -1,212 +0,0 @@
-<!---
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-   http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-
-Opportunistic Containers
-========================
-
-<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
-
-
-<a name="Purpose"></a>Purpose
------------------------------
-
-This document introduces the notion of **opportunistic** container execution, and discusses how opportunistic containers are allocated and executed.
-
-
-<a name="Quick_Guide"></a>Quick Guide
---------------------------------------------------------------------
-
-We start by providing a brief overview of opportunistic containers, including how a user can enable this feature and run a sample job using such containers.
-
-###<a name="Main_Goal"></a>Main Goal
-
-Unlike existing YARN containers that are scheduled in a node only if there are unallocated resources, opportunistic containers can be dispatched to an NM, even if their execution at that node cannot start immediately. In such a case, opportunistic containers will be queued at that NM until resources become available. 
-The main goal of opportunistic container execution is to improve cluster resource utilization, and therefore increase task throughput. Resource utilization and task throughput improvements are more pronounced for workloads that include relatively short tasks (in the order of seconds).
-
-
-###<a name="Enabling_Opportunistic_Containers"></a>Enabling Opportunistic Containers
-
-To enable opportunistic container allocation, the following two properties have to be present in **conf/yarn-site.xml**:
-
-| Property | Description | Default value |
-|:-------- |:----- |:----- |
-| `yarn.resourcemanager.opportunistic-container-allocation.enabled` | Enables opportunistic container allocation. | `false` |
-| `yarn.nodemanager.opportunistic-containers-max-queue-length` | Determines the max number of opportunistic containers that can be queued at an NM. | `0` |
-
-The first parameter above has to be set to `true`. The second one has to be set to a positive value to allow queuing of opportunistic containers at the NM. A value of `10` can be used to start experimenting with opportunistic containers. The optimal value depends on the jobs characteristics, the cluster configuration and the target utilization.
-
-By default, allocation of opportunistic containers is performed centrally through the RM. However, a user can choose to enable distributed allocation of opportunistic containers, which can further improve allocation latency for short tasks. Distributed scheduling can be enabling by setting to `true` the following parameter (note that non-opportunistic containers will continue being scheduled through the RM):
-
-| Property | Description | Default value |
-|:-------- |:----- |:----- |
-| `yarn.nodemanager.distributed-scheduling.enabled` | Enables distributed scheduling. | `false` |
-
-
-###<a name="Running_a_Sample_Job"></a>Running a Sample Job
-
-The following command can be used to run a sample pi map-reduce job, executing 40% of mappers using opportunistic containers (substitute `3.0.0-alpha2-SNAPSHOT` below with the version of Hadoop you are using):
-
-```
-$ hadoop jar hadoop-3.0.0-alpha2-SNAPSHOT/share/hadoop/mapreduce/hadoop-mapreduce-examples-3.0.0-alpha2-SNAPSHOT.jar pi -Dmapreduce.job.num-opportunistic-maps-percent="40" 50 100
-```
-
-By changing the value of `mapreduce.job.num-opportunistic-maps-percent` in the above command, we can specify the percentage of mappers that can be executed through opportunistic containers.
-
-
-###<a name="Opportunistic_Containers_in_Web_UI"></a>Opportunistic Containers in Web UI
-
-When opportunistic container allocation is enabled, the following new columns can be observed in the Nodes page of the Web UI (`rm-address:8088/cluster/nodes`):
-
-* Running Containers (O): number of running opportunistic containers on each node;
-* Mem Used (O): Total memory used by opportunistic containers on each node;
-* VCores Used (O): Total CPU virtual cores used by opportunistic containers on each node;
-* Queued Containers: Number of containers queued at each node.
-
-When clicking on a specific container running on a node, the execution type of the container is also shown.
-
-In the rest of the document, we provide an in-depth description of opportunistic containers, including details about their allocation and execution.
-
-
-Overview <a name="Overview"></a>
---------------------------------
-
-The existing schedulers in YARN (Fair and Capacity Scheduler) allocate containers to a node only if there are unallocated resources at that node at the moment of scheduling the containers. This **guaranteed** type of execution has the advantage that once the AM dispatches a container to a node, the container execution will start immediately, since it is guaranteed that there will be available resources. Moreover, unless fairness or capacity constraints are violated, containers are guaranteed to run to completion without being preempted. 
-
-Although this design offers a more predictable task execution, it has two main drawbacks that can lead to suboptimal cluster resource utilization:
-
-* **Feedback delays.** When a container finishes its execution at a node, the RM gets notified that there are available resources through the next NM-RM heartbeat, then the RM schedules a new container at that node, the AM gets notified through the next AM-RM heartbeat, and finally the AM launches the new container at the node. These delays result in idle node resources, which in turn lead to lower resource utilization, especially when workloads involve tasks whose duration is relatively short.
-* **Allocated vs. utilized resources.** The RM allocates containers based on the *allocated* resources at each node, which might be significantly higher than the actually *utilized* resources (e.g., think of a container for which 4GB memory have been allocated, but only 2GB are being utilized). This lowers effective resource utilization, and can be avoided if the RM takes into account the utilized resources during scheduling. However, this has to be done in a way that allows resources to be reclaimed in case the utilized resources of a running container increase.
-
-To mitigate the above problems, in addition to the existing containers (which we term **guaranteed** containers hereafter), we introduce the notion of **opportunistic** containers. An opportunistic container can be dispatched to an NM, even if there are no available (unallocated) resources for it at the moment of scheduling. In such a case, the opportunistic container will be queued at the NM, waiting for resources to become available for its execution to start. The opportunistic containers are of lower priority than the guaranteed ones, which means that they can be preempted for guaranteed containers to start their execution. Therefore, they can be used to improve cluster resource utilization without impacting the execution of existing guaranteed containers.
-
-An additional advantage of opportunistic containers is that they introduce a notion of **execution priority at the NMs**. For instance, a lower priority job that does not require strict execution guarantees can use opportunistic containers or a mix of container execution types for its tasks.
-
-We have introduced two ways of allocating opportunistic containers: a **centralized** and a **distributed** one. In the centralized scheduling, opportunistic containers are allocated through the YARN RM, whereas in the distributed one, through local schedulers that reside at each NM. Centralized allocation allows for higher quality placement decisions and for implementing more involved sharing policies across applications (e.g., fairness). On the other hand, distributed scheduling can offer faster container allocation, which is useful for short tasks, as it avoids the round-trip to the RM. In both cases, the scheduling of guaranteed containers remains intact and happens through the YARN RM (using the existing Fair or Capacity Scheduler).
-
-Note that in the current implementation, we are allocating containers based on allocated (and not utilized) resources. Therefore, we tackle the "feedback delays" problem mentioned above, but not the "allocated vs. utilized resources" one. There is ongoing work (`YARN-1011`) that employs opportunistic containers to address the latter problem too.
-
-Below, we describe in more detail the [container execution types](#Container_Execution_Types), as well as the [execution](#Execution_of_Opportunistic_Containers) (including the container queuing at the NMs) and [allocation](#Allocation_of_Opportunistic_Containers) of opportunistic containers. Then we discuss how to fine-tune opportunistic containers through some [advanced configuration parameters](#Advanced_Configuration). Finally, we discuss open items for [future work](#Items_for_Future_Work).
-
-
-<a name="Container_Execution_Types"></a>Container Execution Types
------------------------------------------------------------------
-
-We introduce the following two types of containers:
-
-* **Guaranteed containers** correspond to the existing YARN containers. They are allocated by the Fair or Capacity Scheduler, and once dispatched to a node, it is guaranteed that there are available resources for their execution to start immediately. Moreover, these containers run to completion (as long as there are no failures). They can be preempted only in case the scheduler's queue to which they belong, violates fairness or capacity constraints.
-* **Opportunistic containers** are not guaranteed to have resources for their execution to start when they get dispatched to a node. Instead, they might be queued at the NM until resources become available. In case a guaranteed container arrives at a node and there are no resources available for it, one or more opportunistic containers will be preempted to execute the guaranteed one.
-
-When an AM submits its resource requests to the RM, it specifies the type for each container (default is guaranteed), determining the way the container will be [allocated](#Allocation_of_Opportunistic_Containers). Subsequently, when the container is launched by the AM at an NM, its type determines how it will be [executed](#Execution_of_Opportunistic_Containers) by the NM.
-
-
-<a name="Execution_of_Opportunistic_Containers"></a>Execution of Opportunistic Containers
----------------------------------------------------------------------------
-
-When a container arrives at an NM, its execution is determined by the available resources at the NM and the container type. Guaranteed containers start their execution immediately, and if needed, the NM will kill running opportunistic containers to ensure there are sufficient resources for the guaranteed ones to start. On the other hand, opportunistic containers can be queued at the NM, if there are no resources available to start their execution when they arrive at the NM. To enable this, we extended the NM by allowing queuing of containers at each node. The NM monitors the local resources, and when there are sufficient resources available, it starts the execution of the opportunistic container that is at the head of the queue.
-
-In particular, when a container arrives at an NM, localization is performed (i.e., all required resources are downloaded), and then the container moves to a `SCHEDULED` state, in which the container is queued, waiting for its execution to begin:
-
-* If there are available resources, the execution of the container starts immediately, irrespective of its execution type.
-* If there are no available resources:
-    * If the container is guaranteed, we kill as many running opportunistic containers as required for the guaranteed container to be executed, and then start its execution.
-    * If the container is opportunistic, it remains at the queue until resources become available.
-* When a container (guaranteed or opportunistic) finishes its execution and resources get freed up, we examine the queued containers and if there are available resources we start their execution. We pick containers from the queue in a FIFO order.
-
-In the [future work items](#Items_for_Future_Work) below, we discuss different ways of prioritizing task execution (queue reordering) and of killing opportunistic containers to make space for guaranteed ones.
-
-
-<a name="Allocation_of_Opportunistic_Containers"></a>Allocation of Opportunistic Containers
------------------------------------------------------------------------------
-
-As mentioned above, we provide both a centralized and a distributed way of allocating opportunistic containers, which we describe below.
-
-###<a name="Centralized_Allocation"></a>Centralized Allocation
-
-We have introduced a new service at the RM, namely the `OpportunisticContainerAllocatorAMService`, which extends the `ApplicationMasterService`. When the centralized opportunistic allocation is enabled, the resource requests from the AMs are served at the RM side by the `OpportunisticContainerAllocatorAMService`, which splits them into two sets of resource requests: 
-
-* The guaranteed set is forwarded to the existing `ApplicationMasterService` and is subsequently handled by the Fair or Capacity Scheduler.
-* The opportunistic set is handled by the new `OpportunisticContainerAllocator`, which performs the scheduling of opportunistic containers to nodes.
-
-The `OpportunisticContainerAllocator` maintains a list with the [least loaded nodes](#Determining_Nodes_for_Allocation) of the cluster at each moment, and assigns containers to them in a round-robin fashion. Note that in the current implementation, we purposely do not take into account node locality constraints. Since an opportunistic container (unlike the guaranteed ones) might wait at the queue of an NM before its execution starts, it is more important to allocate it at a node that is less loaded (i.e., where queuing delay will be smaller) rather than respect its locality constraints. Moreover, we do not take into account sharing (fairness/capacity) constraints for opportunistic containers at the moment. Support for both locality and sharing constraints can be added in the future if required.
-
-
-###<a name="Distributed_Allocation"></a>Distributed Allocation
-
-In order to enable distributed scheduling of opportunistic containers, we have introduced a new service at each NM, called `AMRMProxyService`. The `AMRMProxyService` implements the `ApplicationMasterService` protocol, and acts as a proxy between the AMs running at that node and the RM. When the `AMRMProxyService` is enabled (through a parameter), we force all AMs running at a particular node to communicate with the `AMRMProxyService` of the same node, instead of going directly to the RM. Moreover, to ensure that the AMs will not talk directly with the RM, when a new AM gets initialized, we replace its `AMRMToken` with a token signed by the `AMRMProxyService`.
-
-A chain of interceptors can be registered with the `AMRMProxyService`. One of these interceptors is the `DistributedScheduler` that is responsible for allocating opportunistic containers in a distributed way, without needing to contact the RM. This modular design makes the `AMRMProxyService` instrumental in other scenarios too, such as YARN federation (`YARN-2915`) or throttling down misbehaving AMs, which can be enabled simply by adding additional interceptors at the interceptor chain.
-
-When distributed opportunistic scheduling is enabled, each AM sends its resource requests to the `AMRMProxyService` running at the same node. The `AMRMProxyService` splits the resource requests into two sets:
-
-* The guaranteed set is forwarded to the RM. In this case the `AMRMProxyService` simply acts as a proxy between the AM and the RM, and the container allocation remains intact (using the Fair or Capacity Scheduler).
-* The opportunistic set is not forwarded to the RM. Instead, it is handled by the `DistributedScheduler` that is running locally at the node. In particular, the `DistributedScheduler` maintains a list with the least loaded nodes in the cluster, and allocates containers to them in a round-robin fashion. The RM informs the `DistributedScheduler` about the least loaded nodes at regular intervals through the NM-RM heartbeats.
-
-The above procedure is similar to the one performed by the `OpportunisticContainerAllocatorAMService` in the case of centralized opportunistic scheduling described above. The main difference is that in the distributed case, the splitting of requests into guaranteed and opportunistic happens locally at the node, and only the guaranteed requests are forwarded to the RM, while the opportunistic ones are handled without contacting the RM.
-
-
-###<a name="Determining_Nodes_for_Allocation"></a>Determining Nodes for Allocation
-
-Each NM informs the RM periodically through the NM-RM heartbeats about the number of running guaranteed and opportunistic containers, as well as the number of queued opportunistic containers. The RM gathers this information from all nodes and determines the least loaded ones.
-
-In the case of centralized allocation of opportunistic containers, this information is immediately available, since the allocation happens centrally. In the case of distributed scheduling, the list with the least loaded nodes is propagated to all NMs (and thus becomes available to the `DistributedSchedulers`) through the heartbeat responses from the RM to the NMs. The number of least loaded nodes sent to the NMs is configurable.
-
-At the moment, we take into account only the number of queued opportunistic containers at each node in order to estimate the time an opportunistic container would have to wait if sent to that node and, thus, determine the least loaded nodes. If the AM provided us with information about the estimated task durations, we could take them into account in order to have better estimates of the queue waiting times.
-
-
-###<a name="Rebalancing_Node_Load"></a>Rebalancing Node Load
-
-Occasionally poor placement choices for opportunistic containers may be made (due to stale queue length estimates), which can lead to load imbalance between nodes. The problem is more pronounced under high cluster load, and also in the case of distributed scheduling (multiple `DistributedSchedulers` may place containers at the same NM, since they do not coordinate with each other). To deal with this load imbalance between the NM queues, we perform load shedding to dynamically re-balance the load between NMs. In particular, while aggregating at the RM the queue time estimates published by each NM, we construct a distribution and find a targeted maximal value for the length of the NM queues (based on the mean and standard deviation of the distribution). Then the RM disseminates this value to the various NMs through the heartbeat responses. Subsequently, using this information, an NM on a node whose queue length is above the threshold discards opportunistic containers to meet this maxi
 mal value. This forces the associated individual AMs to reschedule those containers elsewhere.
-
-
-<a name="Advanced_Configuration"></a>Advanced Configuration
---------------------------------------------------
-
-The main properties for enabling opportunistic container allocation and choosing between centralized and distributed allocation were described in the [quick guide](#Quick_Guide) in the beginning of this document. Here we present more advanced configuration. Note that using default values for those parameters should be sufficient in most cases. All parameters below have to be defined in the **conf/yarn-site.xml** file.
-
-To determine the number of [least loaded nodes](#Determining_Nodes_for_Allocation) that will be used when scheduling opportunistic containers and how often this list will be refreshed, we use the following parameters:
-
-| Property | Description | Default value |
-|:-------- |:----- |:----- |
-| `yarn.resourcemanager.opportunistic-container-allocation.nodes-used` | Number of least loaded nodes to be used by the Opportunistic Container allocator for dispatching containers during container allocation. A higher value can improve load balance in large clusters. | `10` |
-| `yarn.resourcemanager.nm-container-queuing.sorting-nodes-interval-ms` | Frequency for computing least loaded nodes. | `1000` |
-
-
-As discussed in the [node load rebalancing](#Rebalancing_Node_Load) section above, at regular intervals, the RM gathers all NM queue lengths and computes their mean value (`avg`) and standard deviation (`stdev`), as well as the value `avg + k*stdev` (where `k` a float). This value gets propagated through the NM-RM heartbeats to all NMs, who should respect that value by dequeuing containers (if required), as long as their current queue length is between a `queue_min_length` and a `queue_max_length` value (these values are used to avoid dequeuing tasks from very short queues and to aggressively dequeue tasks from long queues, respectively). 
-The parameters `k`, `queue_min_length` and `queue_max_length` can be specified as follows:
-
-| Property | Description | Default value |
-|:-------- |:----- |:----- |
-| `yarn.resourcemanager.nm-container-queuing.queue-limit-stdev` | The `k` parameter. | `1.0f` |
-| `yarn.resourcemanager.nm-container-queuing.min-queue-length` | The `queue_min_length` parameter. | `5` |
-| `yarn.resourcemanager.nm-container-queuing.max-queue-length` | The `queue_max_length` parameter. | `15` |
-
-
-Finally, two more properties can further tune the `AMRMProxyService` in case distributed scheduling is used:
-
-| Property | Description | Default value |
-|:-------- |:----- |:----- |
-| `yarn.nodemanager.amrmproxy.address` | The address/port to which the `AMRMProxyService` is bound to. | `0.0.0.0:8049` |
-| `yarn.nodemanager.amrmproxy.client.thread-count` | The number of threads that are used at each NM for serving the interceptors register to the `AMRMProxyService` by different jobs. | `3` |
-
-
-<a name="Items_for_Future_Work"></a>Items for Future Work
------------------------------------------------
-
-Here we describe multiple ways in which we can extend/enhance the allocation and execution of opportunistic containers. We also provide the JIRAs that track each item.
-
-* **Resource overcommitment** (`YARN-1011`). As already discussed, in order to further improve the cluster resource utilization, we can schedule containers not based on the allocated resources but on the actually utilized ones. When over-committing resources, there is the risk of running out of resources in case we have an increase in the utilized resources of the already running containers. Therefore, opportunistic execution should be used for containers whose allocation goes beyond the capacity of a node. This way, we can choose opportunistic containers to kill for reclaiming resources.
-* **NM Queue reordering** (`YARN-5886`). Instead of executing queued containers in a FIFO order, we can employ reordering strategies that dynamically determine which opportunistic container will be executed next. For example, we can prioritize containers that are expected to be short-running or which belong to applications that are close to completion.
-* **Out of order killing at NMs** (`YARN-5887`). As described above, when we need to free up resources for a guaranteed container to start its execution, we kill opportunistic containers in reverse order of arrival (first the most recently started ones). This might not always be the right decision. For example, we might want to minimize the number of containers killed or to refrain from killing containers of jobs that are very close to completion.
-* **Container pausing** (`YARN-5292`): At the moment we kill opportunistic containers to make room for guaranteed in case of resource contention. In busy clusters this can lower the effective cluster utilization: whenever we kill a running opportunistic container, it has to be restarted, and thus we lose work. To this end, we can instead pause running opportunistic containers. Note that this will require support from the container executor (e.g., the container technology used) and from the application.
-* **Container promotion** (`YARN-5085`). There are cases where changing the execution type of a container during its execution can be beneficial. For instance, an application might submit a container as opportunistic, and when its execution starts, it can request its promotion to a guaranteed container to avoid it getting killed.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b0045e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md.vm
new file mode 100644
index 0000000..7882b87
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md.vm
@@ -0,0 +1,233 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+#set ( $H3 = '###' )
+#set ( $H4 = '####' )
+#set ( $H5 = '#####' )
+
+Opportunistic Containers
+========================
+
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
+
+
+Purpose
+-------
+
+This document introduces the notion of **opportunistic** container execution, and discusses how opportunistic containers are allocated and executed.
+
+
+Quick Guide
+-----------
+
+We start by providing a brief overview of opportunistic containers, including how a user can enable this feature and run a sample job using such containers.
+
+$H3 Main Goal
+
+Unlike existing YARN containers that are scheduled in a node only if there are unallocated resources, opportunistic containers can be dispatched to an NM, even if their execution at that node cannot start immediately. In such a case, opportunistic containers will be queued at that NM until resources become available. 
+The main goal of opportunistic container execution is to improve cluster resource utilization, and therefore increase task throughput. Resource utilization and task throughput improvements are more pronounced for workloads that include relatively short tasks (in the order of seconds).
+
+
+$H3 Enabling Opportunistic Containers
+
+To enable opportunistic container allocation, the following two properties have to be present in **conf/yarn-site.xml**:
+
+| Property | Description | Default value |
+|:-------- |:----- |:----- |
+| `yarn.resourcemanager.opportunistic-container-allocation.enabled` | Enables opportunistic container allocation. | `false` |
+| `yarn.nodemanager.opportunistic-containers-max-queue-length` | Determines the max number of opportunistic containers that can be queued at an NM. | `0` |
+
+The first parameter above has to be set to `true`. The second one has to be set to a positive value to allow queuing of opportunistic containers at the NM. A value of `10` can be used to start experimenting with opportunistic containers. The optimal value depends on the jobs characteristics, the cluster configuration and the target utilization.
+
+By default, allocation of opportunistic containers is performed centrally through the RM. However, a user can choose to enable distributed allocation of opportunistic containers, which can further improve allocation latency for short tasks. Distributed scheduling can be enabling by setting to `true` the following parameter (note that non-opportunistic containers will continue being scheduled through the RM):
+
+| Property | Description | Default value |
+|:-------- |:----- |:----- |
+| `yarn.nodemanager.distributed-scheduling.enabled` | Enables distributed scheduling. | `false` |
+
+In order to submit jobs to a cluster that has AMRMProxy turned on, one must create a separate set of configs for the client from which jobs will be submitted. In these, the **conf/yarn-site.xml** should have the following additional configurations:
+
+| Property | Value | Description |
+|:-------- |:----- |:----- |
+| `yarn.resourcemanger.scheduler.address` | `localhost:8049` | Redirects jobs to the Node Manager's AMRMProxy port.|
+
+
+$H3 Running a Sample Job
+
+$H4 MapReduce PI
+
+The following command can be used to run a sample pi map-reduce job, executing 40% of mappers using opportunistic containers:
+
+```
+$ hadoop jar share/hadoop/mapreduce/hadoop-mapreduce-examples-${project.version}.jar pi -Dmapreduce.job.num-opportunistic-maps-percent="40" 50 100
+```
+
+By changing the value of `mapreduce.job.num-opportunistic-maps-percent` in the above command, we can specify the percentage of mappers that can be executed through opportunistic containers.
+
+$H4 Distributed Shell
+
+Another sample job is the distributed shell, it allows us to run a given shell command on a set of containers. The following command can be used to run `sleep 10` command in 10 opportunistic containers:
+
+```
+$ yarn org.apache.hadoop.yarn.applications.distributedshell.Client -jar share/hadoop/yarn/hadoop-yarn-applications-distributedshell-${project.version}.jar.jar -shell_command sleep -shell_args 10 -num_containers 10 -container_type OPPORTUNISTIC
+```
+
+By change the value of `container_type` to `OPPORTUNISTIC` or `GUARANTEED` in the above command, we can specify the tasks to be running in opportunistic or guaranteed containers. The default type is `GUARANTEED`.
+
+$H3 Opportunistic Containers in Web UI
+
+When opportunistic container allocation is enabled, the following new columns can be observed in the Nodes page of the Web UI (`rm-address:8088/cluster/nodes`):
+
+* Running Containers (O): number of running opportunistic containers on each node;
+* Mem Used (O): Total memory used by opportunistic containers on each node;
+* VCores Used (O): Total CPU virtual cores used by opportunistic containers on each node;
+* Queued Containers: Number of containers queued at each node.
+
+When clicking on a specific container running on a node, the execution type of the container is also shown.
+
+In the rest of the document, we provide an in-depth description of opportunistic containers, including details about their allocation and execution.
+
+
+Overview
+--------
+
+The existing schedulers in YARN (Fair and Capacity Scheduler) allocate containers to a node only if there are unallocated resources at that node at the moment of scheduling the containers. This **guaranteed** type of execution has the advantage that once the AM dispatches a container to a node, the container execution will start immediately, since it is guaranteed that there will be available resources. Moreover, unless fairness or capacity constraints are violated, containers are guaranteed to run to completion without being preempted. 
+
+Although this design offers a more predictable task execution, it has two main drawbacks that can lead to suboptimal cluster resource utilization:
+
+* **Feedback delays.** When a container finishes its execution at a node, the RM gets notified that there are available resources through the next NM-RM heartbeat, then the RM schedules a new container at that node, the AM gets notified through the next AM-RM heartbeat, and finally the AM launches the new container at the node. These delays result in idle node resources, which in turn lead to lower resource utilization, especially when workloads involve tasks whose duration is relatively short.
+* **Allocated vs. utilized resources.** The RM allocates containers based on the *allocated* resources at each node, which might be significantly higher than the actually *utilized* resources (e.g., think of a container for which 4GB memory have been allocated, but only 2GB are being utilized). This lowers effective resource utilization, and can be avoided if the RM takes into account the utilized resources during scheduling. However, this has to be done in a way that allows resources to be reclaimed in case the utilized resources of a running container increase.
+
+To mitigate the above problems, in addition to the existing containers (which we term **guaranteed** containers hereafter), we introduce the notion of **opportunistic** containers. An opportunistic container can be dispatched to an NM, even if there are no available (unallocated) resources for it at the moment of scheduling. In such a case, the opportunistic container will be queued at the NM, waiting for resources to become available for its execution to start. The opportunistic containers are of lower priority than the guaranteed ones, which means that they can be preempted for guaranteed containers to start their execution. Therefore, they can be used to improve cluster resource utilization without impacting the execution of existing guaranteed containers.
+
+An additional advantage of opportunistic containers is that they introduce a notion of **execution priority at the NMs**. For instance, a lower priority job that does not require strict execution guarantees can use opportunistic containers or a mix of container execution types for its tasks.
+
+We have introduced two ways of allocating opportunistic containers: a **centralized** and a **distributed** one. In the centralized scheduling, opportunistic containers are allocated through the YARN RM, whereas in the distributed one, through local schedulers that reside at each NM. Centralized allocation allows for higher quality placement decisions and for implementing more involved sharing policies across applications (e.g., fairness). On the other hand, distributed scheduling can offer faster container allocation, which is useful for short tasks, as it avoids the round-trip to the RM. In both cases, the scheduling of guaranteed containers remains intact and happens through the YARN RM (using the existing Fair or Capacity Scheduler).
+
+Note that in the current implementation, we are allocating containers based on allocated (and not utilized) resources. Therefore, we tackle the "feedback delays" problem mentioned above, but not the "allocated vs. utilized resources" one. There is ongoing work (`YARN-1011`) that employs opportunistic containers to address the latter problem too.
+
+Below, we describe in more detail the [container execution types](#Container_Execution_Types), as well as the [execution](#Execution_of_Opportunistic_Containers) (including the container queuing at the NMs) and [allocation](#Allocation_of_Opportunistic_Containers) of opportunistic containers. Then we discuss how to fine-tune opportunistic containers through some [advanced configuration parameters](#Advanced_Configuration). Finally, we discuss open items for [future work](#Items_for_Future_Work).
+
+
+Container Execution Types
+-------------------------
+
+We introduce the following two types of containers:
+
+* **Guaranteed containers** correspond to the existing YARN containers. They are allocated by the Fair or Capacity Scheduler, and once dispatched to a node, it is guaranteed that there are available resources for their execution to start immediately. Moreover, these containers run to completion (as long as there are no failures). They can be preempted only in case the scheduler's queue to which they belong, violates fairness or capacity constraints.
+* **Opportunistic containers** are not guaranteed to have resources for their execution to start when they get dispatched to a node. Instead, they might be queued at the NM until resources become available. In case a guaranteed container arrives at a node and there are no resources available for it, one or more opportunistic containers will be preempted to execute the guaranteed one.
+
+When an AM submits its resource requests to the RM, it specifies the type for each container (default is guaranteed), determining the way the container will be [allocated](#Allocation_of_Opportunistic_Containers). Subsequently, when the container is launched by the AM at an NM, its type determines how it will be [executed](#Execution_of_Opportunistic_Containers) by the NM.
+
+
+Execution of Opportunistic Containers
+-------------------------------------
+
+When a container arrives at an NM, its execution is determined by the available resources at the NM and the container type. Guaranteed containers start their execution immediately, and if needed, the NM will kill running opportunistic containers to ensure there are sufficient resources for the guaranteed ones to start. On the other hand, opportunistic containers can be queued at the NM, if there are no resources available to start their execution when they arrive at the NM. To enable this, we extended the NM by allowing queuing of containers at each node. The NM monitors the local resources, and when there are sufficient resources available, it starts the execution of the opportunistic container that is at the head of the queue.
+
+In particular, when a container arrives at an NM, localization is performed (i.e., all required resources are downloaded), and then the container moves to a `SCHEDULED` state, in which the container is queued, waiting for its execution to begin:
+
+* If there are available resources, the execution of the container starts immediately, irrespective of its execution type.
+* If there are no available resources:
+    * If the container is guaranteed, we kill as many running opportunistic containers as required for the guaranteed container to be executed, and then start its execution.
+    * If the container is opportunistic, it remains at the queue until resources become available.
+* When a container (guaranteed or opportunistic) finishes its execution and resources get freed up, we examine the queued containers and if there are available resources we start their execution. We pick containers from the queue in a FIFO order.
+
+In the [future work items](#Items_for_Future_Work) below, we discuss different ways of prioritizing task execution (queue reordering) and of killing opportunistic containers to make space for guaranteed ones.
+
+
+Allocation of Opportunistic Containers
+--------------------------------------
+
+As mentioned above, we provide both a centralized and a distributed way of allocating opportunistic containers, which we describe below.
+
+$H3 Centralized Allocation
+
+We have introduced a new service at the RM, namely the `OpportunisticContainerAllocatorAMService`, which extends the `ApplicationMasterService`. When the centralized opportunistic allocation is enabled, the resource requests from the AMs are served at the RM side by the `OpportunisticContainerAllocatorAMService`, which splits them into two sets of resource requests: 
+
+* The guaranteed set is forwarded to the existing `ApplicationMasterService` and is subsequently handled by the Fair or Capacity Scheduler.
+* The opportunistic set is handled by the new `OpportunisticContainerAllocator`, which performs the scheduling of opportunistic containers to nodes.
+
+The `OpportunisticContainerAllocator` maintains a list with the [least loaded nodes](#Determining_Nodes_for_Allocation) of the cluster at each moment, and assigns containers to them in a round-robin fashion. Note that in the current implementation, we purposely do not take into account node locality constraints. Since an opportunistic container (unlike the guaranteed ones) might wait at the queue of an NM before its execution starts, it is more important to allocate it at a node that is less loaded (i.e., where queuing delay will be smaller) rather than respect its locality constraints. Moreover, we do not take into account sharing (fairness/capacity) constraints for opportunistic containers at the moment. Support for both locality and sharing constraints can be added in the future if required.
+
+
+$H3 Distributed Allocation
+
+In order to enable distributed scheduling of opportunistic containers, we have introduced a new service at each NM, called `AMRMProxyService`. The `AMRMProxyService` implements the `ApplicationMasterService` protocol, and acts as a proxy between the AMs running at that node and the RM. When the `AMRMProxyService` is enabled (through a parameter), we force all AMs running at a particular node to communicate with the `AMRMProxyService` of the same node, instead of going directly to the RM. Moreover, to ensure that the AMs will not talk directly with the RM, when a new AM gets initialized, we replace its `AMRMToken` with a token signed by the `AMRMProxyService`.
+
+A chain of interceptors can be registered with the `AMRMProxyService`. One of these interceptors is the `DistributedScheduler` that is responsible for allocating opportunistic containers in a distributed way, without needing to contact the RM. This modular design makes the `AMRMProxyService` instrumental in other scenarios too, such as YARN federation (`YARN-2915`) or throttling down misbehaving AMs, which can be enabled simply by adding additional interceptors at the interceptor chain.
+
+When distributed opportunistic scheduling is enabled, each AM sends its resource requests to the `AMRMProxyService` running at the same node. The `AMRMProxyService` splits the resource requests into two sets:
+
+* The guaranteed set is forwarded to the RM. In this case the `AMRMProxyService` simply acts as a proxy between the AM and the RM, and the container allocation remains intact (using the Fair or Capacity Scheduler).
+* The opportunistic set is not forwarded to the RM. Instead, it is handled by the `DistributedScheduler` that is running locally at the node. In particular, the `DistributedScheduler` maintains a list with the least loaded nodes in the cluster, and allocates containers to them in a round-robin fashion. The RM informs the `DistributedScheduler` about the least loaded nodes at regular intervals through the NM-RM heartbeats.
+
+The above procedure is similar to the one performed by the `OpportunisticContainerAllocatorAMService` in the case of centralized opportunistic scheduling described above. The main difference is that in the distributed case, the splitting of requests into guaranteed and opportunistic happens locally at the node, and only the guaranteed requests are forwarded to the RM, while the opportunistic ones are handled without contacting the RM.
+
+
+$H3 Determining Nodes for Allocation
+
+Each NM informs the RM periodically through the NM-RM heartbeats about the number of running guaranteed and opportunistic containers, as well as the number of queued opportunistic containers. The RM gathers this information from all nodes and determines the least loaded ones.
+
+In the case of centralized allocation of opportunistic containers, this information is immediately available, since the allocation happens centrally. In the case of distributed scheduling, the list with the least loaded nodes is propagated to all NMs (and thus becomes available to the `DistributedSchedulers`) through the heartbeat responses from the RM to the NMs. The number of least loaded nodes sent to the NMs is configurable.
+
+At the moment, we take into account only the number of queued opportunistic containers at each node in order to estimate the time an opportunistic container would have to wait if sent to that node and, thus, determine the least loaded nodes. If the AM provided us with information about the estimated task durations, we could take them into account in order to have better estimates of the queue waiting times.
+
+
+$H3 Rebalancing Node Load
+
+Occasionally poor placement choices for opportunistic containers may be made (due to stale queue length estimates), which can lead to load imbalance between nodes. The problem is more pronounced under high cluster load, and also in the case of distributed scheduling (multiple `DistributedSchedulers` may place containers at the same NM, since they do not coordinate with each other). To deal with this load imbalance between the NM queues, we perform load shedding to dynamically re-balance the load between NMs. In particular, while aggregating at the RM the queue time estimates published by each NM, we construct a distribution and find a targeted maximal value for the length of the NM queues (based on the mean and standard deviation of the distribution). Then the RM disseminates this value to the various NMs through the heartbeat responses. Subsequently, using this information, an NM on a node whose queue length is above the threshold discards opportunistic containers to meet this maxi
 mal value. This forces the associated individual AMs to reschedule those containers elsewhere.
+
+
+Advanced Configuration
+----------------------
+
+The main properties for enabling opportunistic container allocation and choosing between centralized and distributed allocation were described in the [quick guide](#Quick_Guide) in the beginning of this document. Here we present more advanced configuration. Note that using default values for those parameters should be sufficient in most cases. All parameters below have to be defined in the **conf/yarn-site.xml** file.
+
+To determine the number of [least loaded nodes](#Determining_Nodes_for_Allocation) that will be used when scheduling opportunistic containers and how often this list will be refreshed, we use the following parameters:
+
+| Property | Description | Default value |
+|:-------- |:----- |:----- |
+| `yarn.resourcemanager.opportunistic-container-allocation.nodes-used` | Number of least loaded nodes to be used by the Opportunistic Container allocator for dispatching containers during container allocation. A higher value can improve load balance in large clusters. | `10` |
+| `yarn.resourcemanager.nm-container-queuing.sorting-nodes-interval-ms` | Frequency for computing least loaded nodes. | `1000` |
+
+
+As discussed in the [node load rebalancing](#Rebalancing_Node_Load) section above, at regular intervals, the RM gathers all NM queue lengths and computes their mean value (`avg`) and standard deviation (`stdev`), as well as the value `avg + k*stdev` (where `k` a float). This value gets propagated through the NM-RM heartbeats to all NMs, who should respect that value by dequeuing containers (if required), as long as their current queue length is between a `queue_min_length` and a `queue_max_length` value (these values are used to avoid dequeuing tasks from very short queues and to aggressively dequeue tasks from long queues, respectively). 
+The parameters `k`, `queue_min_length` and `queue_max_length` can be specified as follows:
+
+| Property | Description | Default value |
+|:-------- |:----- |:----- |
+| `yarn.resourcemanager.nm-container-queuing.queue-limit-stdev` | The `k` parameter. | `1.0f` |
+| `yarn.resourcemanager.nm-container-queuing.min-queue-length` | The `queue_min_length` parameter. | `5` |
+| `yarn.resourcemanager.nm-container-queuing.max-queue-length` | The `queue_max_length` parameter. | `15` |
+
+
+Finally, two more properties can further tune the `AMRMProxyService` in case distributed scheduling is used:
+
+| Property | Description | Default value |
+|:-------- |:----- |:----- |
+| `yarn.nodemanager.amrmproxy.address` | The address/port to which the `AMRMProxyService` is bound to. | `0.0.0.0:8049` |
+| `yarn.nodemanager.amrmproxy.client.thread-count` | The number of threads that are used at each NM for serving the interceptors register to the `AMRMProxyService` by different jobs. | `3` |
+
+
+Items for Future Work
+---------------------
+
+Here we describe multiple ways in which we can extend/enhance the allocation and execution of opportunistic containers. We also provide the JIRAs that track each item.
+
+* **Resource overcommitment** (`YARN-1011`). As already discussed, in order to further improve the cluster resource utilization, we can schedule containers not based on the allocated resources but on the actually utilized ones. When over-committing resources, there is the risk of running out of resources in case we have an increase in the utilized resources of the already running containers. Therefore, opportunistic execution should be used for containers whose allocation goes beyond the capacity of a node. This way, we can choose opportunistic containers to kill for reclaiming resources.
+* **NM Queue reordering** (`YARN-5886`). Instead of executing queued containers in a FIFO order, we can employ reordering strategies that dynamically determine which opportunistic container will be executed next. For example, we can prioritize containers that are expected to be short-running or which belong to applications that are close to completion.
+* **Out of order killing at NMs** (`YARN-5887`). As described above, when we need to free up resources for a guaranteed container to start its execution, we kill opportunistic containers in reverse order of arrival (first the most recently started ones). This might not always be the right decision. For example, we might want to minimize the number of containers killed or to refrain from killing containers of jobs that are very close to completion.
+* **Container pausing** (`YARN-5292`): At the moment we kill opportunistic containers to make room for guaranteed in case of resource contention. In busy clusters this can lower the effective cluster utilization: whenever we kill a running opportunistic container, it has to be restarted, and thus we lose work. To this end, we can instead pause running opportunistic containers. Note that this will require support from the container executor (e.g., the container technology used) and from the application.
+* **Container promotion** (`YARN-5085`). There are cases where changing the execution type of a container during its execution can be beneficial. For instance, an application might submit a container as opportunistic, and when its execution starts, it can request its promotion to a guaranteed container to avoid it getting killed.


---------------------------------------------------------------------
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: HDFS-12638. Delete copy-on-truncate block along with the original block, when deleting a file being truncated. Contributed by Konstantin Shvachko.

Posted by ae...@apache.org.
HDFS-12638. Delete copy-on-truncate block along with the original block, when deleting a file being truncated. Contributed by Konstantin Shvachko.

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

Branch: refs/heads/HDFS-7240
Commit: 60fd0d7fd73198fd610e59d1a4cd007c5fcc7205
Parents: a63d19d
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Thu Nov 30 18:18:09 2017 -0800
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Thu Nov 30 18:18:28 2017 -0800

----------------------------------------------------------------------
 .../hadoop/hdfs/server/namenode/INode.java      | 14 +++++++
 .../hdfs/server/namenode/TestFileTruncate.java  | 41 ++++++++++++++++++++
 2 files changed, 55 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/60fd0d7f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
index 34bfe10..1682a30 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
@@ -33,9 +33,11 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
@@ -1058,6 +1060,18 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
       assert toDelete != null : "toDelete is null";
       toDelete.delete();
       toDeleteList.add(toDelete);
+      // If the file is being truncated
+      // the copy-on-truncate block should also be collected for deletion
+      BlockUnderConstructionFeature uc = toDelete.getUnderConstructionFeature();
+      if(uc == null) {
+        return;
+      }
+      Block truncateBlock = uc.getTruncateBlock();
+      if(truncateBlock == null || truncateBlock.equals(toDelete)) {
+        return;
+      }
+      assert truncateBlock instanceof BlockInfo : "should be BlockInfo";
+      addDeleteBlock((BlockInfo) truncateBlock);
     }
 
     public void addUpdateReplicationFactor(BlockInfo block, short targetRepl) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60fd0d7f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index d4215e8..51a94e7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -60,6 +60,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.FsDatasetTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
+import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
@@ -1155,6 +1156,46 @@ public class TestFileTruncate {
     fs.delete(parent, true);
   }
 
+  /**
+   * While rolling upgrade is in-progress the test truncates a file
+   * such that copy-on-truncate is triggered, then deletes the file,
+   * and makes sure that no blocks involved in truncate are hanging around.
+   */
+  @Test
+  public void testTruncateWithRollingUpgrade() throws Exception {
+    final DFSAdmin dfsadmin = new DFSAdmin(cluster.getConfiguration(0));
+    DistributedFileSystem dfs = cluster.getFileSystem();
+    //start rolling upgrade
+    dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    int status = dfsadmin.run(new String[]{"-rollingUpgrade", "prepare"});
+    assertEquals("could not prepare for rolling upgrade", 0, status);
+    dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+
+    Path dir = new Path("/testTruncateWithRollingUpgrade");
+    fs.mkdirs(dir);
+    final Path p = new Path(dir, "file");
+    final byte[] data = new byte[3];
+    ThreadLocalRandom.current().nextBytes(data);
+    writeContents(data, data.length, p);
+
+    assertEquals("block num should 1", 1,
+        cluster.getNamesystem().getFSDirectory().getBlockManager()
+            .getTotalBlocks());
+
+    final boolean isReady = fs.truncate(p, 2);
+    assertFalse("should be copy-on-truncate", isReady);
+    assertEquals("block num should 2", 2,
+        cluster.getNamesystem().getFSDirectory().getBlockManager()
+            .getTotalBlocks());
+    fs.delete(p, true);
+
+    assertEquals("block num should 0", 0,
+        cluster.getNamesystem().getFSDirectory().getBlockManager()
+            .getTotalBlocks());
+    status = dfsadmin.run(new String[]{"-rollingUpgrade", "finalize"});
+    assertEquals("could not finalize rolling upgrade", 0, status);
+  }
+
   static void writeContents(byte[] contents, int fileLength, Path p)
       throws IOException {
     FSDataOutputStream out = fs.create(p, true, BLOCK_SIZE, REPLICATION,


---------------------------------------------------------------------
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-6907. Node information page in the old web UI should report resource types. Contributed by Gergely Novák.

Posted by ae...@apache.org.
YARN-6907. Node information page in the old web UI should report resource types. Contributed by Gergely Novák.


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

Branch: refs/heads/HDFS-7240
Commit: 81f6e46b2fb54659a08864677ad14f80fe4e452d
Parents: 30f2646
Author: Sunil G <su...@apache.org>
Authored: Mon Dec 4 11:27:23 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Mon Dec 4 11:27:23 2017 +0530

----------------------------------------------------------------------
 .../yarn/server/nodemanager/webapp/NodePage.java      |  2 ++
 .../yarn/server/nodemanager/webapp/dao/NodeInfo.java  |  9 +++++++++
 .../server/nodemanager/webapp/TestNMWebServices.java  | 14 ++++++++++----
 3 files changed, 21 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/81f6e46b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NodePage.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/NodePage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NodePage.java
index 7005f41..ae9b92d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NodePage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NodePage.java
@@ -75,6 +75,8 @@ public class NodePage extends NMView {
               info.isPmemCheckEnabled())
           .__("Total VCores allocated for Containers",
               String.valueOf(info.getTotalVCoresAllocated()))
+          .__("Resource types",
+              info.getResourceTypes())
           .__("NodeHealthyStatus",
               info.getHealthStatus())
           .__("LastNodeHealthTime", new Date(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81f6e46b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/NodeInfo.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/NodeInfo.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/NodeInfo.java
index 32e39cf..33e2de6 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/NodeInfo.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/NodeInfo.java
@@ -22,11 +22,13 @@ import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.nodemanager.ResourceView;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 
 @XmlRootElement
 @XmlAccessorType(XmlAccessType.FIELD)
@@ -41,6 +43,7 @@ public class NodeInfo {
   protected boolean vmemCheckEnabled;
   protected boolean pmemCheckEnabled;
   protected long lastNodeUpdateTime;
+  protected String resourceTypes;
   protected boolean nodeHealthy;
   protected String nodeManagerVersion;
   protected String nodeManagerBuildVersion;
@@ -67,6 +70,8 @@ public class NodeInfo {
     this.pmemCheckEnabled = resourceView.isPmemCheckEnabled();
     this.totalVCoresAllocatedContainers = resourceView
         .getVCoresAllocatedForContainers();
+    this.resourceTypes = StringUtils.join(", ",
+        ResourceUtils.getResourcesTypeInfo());
     this.nodeHealthy = context.getNodeHealthStatus().getIsNodeHealthy();
     this.lastNodeUpdateTime = context.getNodeHealthStatus()
         .getLastHealthReportTime();
@@ -146,6 +151,10 @@ public class NodeInfo {
     return this.pmemCheckEnabled;
   }
 
+  public String getResourceTypes() {
+    return this.resourceTypes;
+  }
+
   public long getNMStartupTime() {
     return nmStartupTime;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81f6e46b/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 45eb085..39e403d 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
@@ -745,14 +745,15 @@ public class TestNMWebServices extends JerseyTestBase {
           WebServicesTestUtils.getXmlString(element,
               "nodeManagerVersionBuiltOn"), WebServicesTestUtils.getXmlString(
               element, "nodeManagerBuildVersion"),
-          WebServicesTestUtils.getXmlString(element, "nodeManagerVersion"));
+          WebServicesTestUtils.getXmlString(element, "nodeManagerVersion"),
+          WebServicesTestUtils.getXmlString(element, "resourceTypes"));
     }
   }
 
   public void verifyNodeInfo(JSONObject json) throws JSONException, Exception {
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject info = json.getJSONObject("nodeInfo");
-    assertEquals("incorrect number of elements", 17, info.length());
+    assertEquals("incorrect number of elements", 18, info.length());
     verifyNodeInfoGeneric(info.getString("id"), info.getString("healthReport"),
         info.getLong("totalVmemAllocatedContainersMB"),
         info.getLong("totalPmemAllocatedContainersMB"),
@@ -764,7 +765,9 @@ public class TestNMWebServices extends JerseyTestBase {
         info.getString("hadoopBuildVersion"), info.getString("hadoopVersion"),
         info.getString("nodeManagerVersionBuiltOn"),
         info.getString("nodeManagerBuildVersion"),
-        info.getString("nodeManagerVersion"));
+        info.getString("nodeManagerVersion"),
+        info.getString("resourceTypes")
+        );
 
   }
 
@@ -775,7 +778,8 @@ public class TestNMWebServices extends JerseyTestBase {
       long lastNodeUpdateTime, Boolean nodeHealthy, String nodeHostName,
       String hadoopVersionBuiltOn, String hadoopBuildVersion,
       String hadoopVersion, String resourceManagerVersionBuiltOn,
-      String resourceManagerBuildVersion, String resourceManagerVersion) {
+      String resourceManagerBuildVersion, String resourceManagerVersion,
+      String resourceTypes) {
 
     WebServicesTestUtils.checkStringMatch("id", "testhost.foo.com:8042", id);
     WebServicesTestUtils.checkStringMatch("healthReport", "Healthy",
@@ -807,6 +811,8 @@ public class TestNMWebServices extends JerseyTestBase {
         YarnVersionInfo.getBuildVersion(), resourceManagerBuildVersion);
     WebServicesTestUtils.checkStringMatch("resourceManagerVersion",
         YarnVersionInfo.getVersion(), resourceManagerVersion);
+
+    assertEquals("memory-mb (unit=Mi), vcores", resourceTypes);
   }
 
   private String getLogContext(String fullMessage) {


---------------------------------------------------------------------
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-7594. TestNMWebServices#testGetNMResourceInfo fails on trunk. Contributed by Gergely Novák.

Posted by ae...@apache.org.
YARN-7594. TestNMWebServices#testGetNMResourceInfo fails on trunk. Contributed by Gergely Novák.


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

Branch: refs/heads/HDFS-7240
Commit: 30f2646b159d0d8d192e33d38434b7056855b468
Parents: 42307e3
Author: Sunil G <su...@apache.org>
Authored: Mon Dec 4 10:45:07 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Mon Dec 4 10:45:07 2017 +0530

----------------------------------------------------------------------
 .../server/nodemanager/webapp/TestNMWebServices.java    | 12 ++++--------
 1 file changed, 4 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/30f2646b/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 171834e..45eb085 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
@@ -456,18 +456,18 @@ public class TestNMWebServices extends JerseyTestBase {
     assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
         response.getType().toString());
 
-    // Access resource-2 should fail (null NMResourceInfo returned).
+    // Access resource-2 should fail (empty NMResourceInfo returned).
     JSONObject json = response.getEntity(JSONObject.class);
-    assertIncludesException(json);
+    Assert.assertEquals(0, json.length());
 
-    // Access resource-3 should fail (unkown plugin)
+    // Access resource-3 should fail (unknown plugin)
     response = r.path("ws").path("v1").path("node").path(
         "resources").path("resource-3").accept(MediaType.APPLICATION_JSON).get(
         ClientResponse.class);
     assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
         response.getType().toString());
     json = response.getEntity(JSONObject.class);
-    assertIncludesException(json);
+    Assert.assertEquals(0, json.length());
 
     // Access resource-1 should success
     response = r.path("ws").path("v1").path("node").path(
@@ -537,10 +537,6 @@ public class TestNMWebServices extends JerseyTestBase {
     Assert.assertEquals(2, json.getJSONArray("assignedGpuDevices").length());
   }
 
-  private void assertIncludesException(JSONObject json) {
-    Assert.assertTrue(json.has("RemoteException"));
-  }
-
   private void testContainerLogs(WebResource r, ContainerId containerId)
       throws IOException {
     final String containerIdStr = containerId.toString();


---------------------------------------------------------------------
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: YARN-7092. Render application specific log under application tab in new YARN UI. Contributed by Akhil PB.

Posted by ae...@apache.org.
YARN-7092. Render application specific log under application tab in 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/99ccca34
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/99ccca34
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/99ccca34

Branch: refs/heads/HDFS-7240
Commit: 99ccca341f3669b801428dea0acdba597f34c668
Parents: f9f317b
Author: Sunil G <su...@apache.org>
Authored: Tue Dec 5 19:41:07 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Tue Dec 5 19:41:07 2017 +0530

----------------------------------------------------------------------
 .../src/main/webapp/app/adapters/yarn-log.js    |  39 ++++
 .../webapp/app/components/collapsible-panel.js  |  64 ++++++
 .../main/webapp/app/components/timeline-view.js |   6 +-
 .../webapp/app/controllers/yarn-app-attempt.js  |   2 +-
 .../webapp/app/controllers/yarn-app/logs.js     | 204 +++++++++++++++++++
 .../src/main/webapp/app/models/yarn-log.js      |  27 +++
 .../src/main/webapp/app/router.js               |   1 +
 .../main/webapp/app/routes/yarn-app-attempt.js  |   4 +-
 .../main/webapp/app/routes/yarn-app/attempts.js |   4 +-
 .../src/main/webapp/app/routes/yarn-app/logs.js |  47 +++++
 .../src/main/webapp/app/serializers/yarn-log.js |  48 +++++
 .../templates/components/collapsible-panel.hbs  |  21 ++
 .../app/templates/components/timeline-view.hbs  |  66 +++---
 .../webapp/app/templates/yarn-app-attempt.hbs   |  21 +-
 .../src/main/webapp/app/templates/yarn-app.hbs  |   3 +
 .../webapp/app/templates/yarn-app/attempts.hbs  |  13 ++
 .../main/webapp/app/templates/yarn-app/logs.hbs | 132 ++++++++++++
 .../src/main/webapp/bower-shrinkwrap.json       |  12 +-
 .../components/collapsible-panel-test.js        |  43 ++++
 .../webapp/tests/unit/adapters/yarn-log-test.js |  30 +++
 .../unit/controllers/yarn-app/logs-test.js      |  30 +++
 .../webapp/tests/unit/models/yarn-log-test.js   |  30 +++
 .../tests/unit/routes/yarn-app/logs-test.js     |  29 +++
 .../tests/unit/serializers/yarn-log-test.js     |  33 +++
 24 files changed, 857 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-log.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-log.js
new file mode 100644
index 0000000..df29b71
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-log.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 AbstractAdapter from './abstract';
+
+export default AbstractAdapter.extend({
+  address: "timelineWebAddress",
+  // restNameSpace: "timelineV2", // Use ATSv2 when it supports log APIs.
+  restNameSpace: "timeline", //Using ATSv1.5 now, would be supported by ATSv2 very soon.
+  serverName: "ATS",
+
+  urlForQuery(query/*, modelName*/) {
+    var url = this._buildURL();
+    var containerId = query['containerId'];
+    delete query.containerId;
+    return url + '/containers/' + containerId + '/logs';
+  },
+
+  fetchLogFileContent(containerId, logFile) {
+    var url = this._buildURL();
+    url = url + '/containers/' + containerId + '/logs/' + logFile;
+    return Ember.$.ajax({url: url, type: 'GET', dataType: 'text'});
+  }
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/collapsible-panel.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/collapsible-panel.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/collapsible-panel.js
new file mode 100644
index 0000000..6a40f8f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/collapsible-panel.js
@@ -0,0 +1,64 @@
+/**
+ * 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.Component.extend({
+  classNames: ['pull-right'],
+
+  targetId: '',
+  initialClosedState: false,
+
+  didInsertElement() {
+    if (!this.get('targetId')) {
+      this.$('.toggle_switch').hide();
+    }
+    if (this.get('targetId') && this.get('initialClosedState')) {
+      this.$('.toggle_switch').show();
+      this.toggleToggleSwitchArrow();
+      Ember.$('#' + this.get('targetId')).removeClass('panel-collapsed').show();
+    }
+  },
+
+  toggleToggleSwitchArrow() {
+    let $toggleArrow = this.$('.toggle_switch').find('span');
+    if ($toggleArrow.hasClass('glyphicon-chevron-up')) {
+      $toggleArrow.removeClass('glyphicon-chevron-up').addClass('glyphicon-chevron-down');
+    } else {
+      $toggleArrow.removeClass('glyphicon-chevron-down').addClass('glyphicon-chevron-up');
+    }
+  },
+
+  toggleCollapsiblePanel() {
+    let $collapsiblePanel = Ember.$('#' + this.get('targetId'));
+    if ($collapsiblePanel.hasClass('panel-collapsed')) {
+      $collapsiblePanel.removeClass('panel-collapsed');
+      $collapsiblePanel.slideDown();
+    } else {
+      $collapsiblePanel.addClass('panel-collapsed');
+      $collapsiblePanel.slideUp();
+    }
+  },
+
+  actions: {
+    togglePanelCollapse() {
+      this.toggleToggleSwitchArrow();
+      this.toggleCollapsiblePanel();
+    }
+  }
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/timeline-view.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/timeline-view.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/timeline-view.js
index 865fe52..420d793 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/timeline-view.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/timeline-view.js
@@ -472,9 +472,5 @@ export default Ember.Component.extend({
       prop = 'http://' + prop;
     }
     return prop;
-  },
-
-  isDataEmpty: Ember.computed(function() {
-    return this.modelArr.length === 0;
-  })
+  }
 });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/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 1121a84..504b66e 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
@@ -22,7 +22,7 @@ export default Ember.Controller.extend({
   queryParams: ["service"],
   service: undefined,
 
-  breadcrumbs: Ember.computed("model.attempt.appId", function () {
+  breadcrumbs: Ember.computed("model.attempt.appId", "model.attempt.id", function () {
     var appId = this.get("model.attempt.appId");
     var attemptId = this.get("model.attempt.id");
     var serviceName = this.get('service');

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/logs.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/logs.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/logs.js
new file mode 100644
index 0000000..c996f05
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/logs.js
@@ -0,0 +1,204 @@
+/**
+ * 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({
+  queryParams: ["service"],
+  service: undefined,
+
+  selectedAttemptId: '',
+  attemptContainerList: null,
+  selectedContainerId: '',
+  selectedLogFileName: '',
+  containerLogFiles: null,
+  selectedLogFileContent: '',
+
+  _isLoadingTopPanel: false,
+  _isLoadingBottomPanel: false,
+
+  actions: {
+    showContainersForAttemptId(attemptId) {
+      this.set('selectedAttemptId', '');
+      if (attemptId) {
+        this.set('_isLoadingTopPanel', true);
+        this.set('selectedAttemptId', attemptId);
+        this.fetchContainersForAttemptId(attemptId).then((hash) => {
+          let containers = null;
+          if (hash.rmContainers.get('length') > 0 && hash.rmContainers.get('content')) {
+            containers = (containers || []).concat(hash.rmContainers.get('content'));
+          }
+          if (hash.tsContainers.get('length') > 0 && hash.tsContainers.get('content')) {
+            containers = (containers || []).concat(hash.tsContainers.get('content'));
+          }
+          this.set('attemptContainerList', containers);
+        }).finally(() => {
+          this.set('_isLoadingTopPanel', false);
+        });
+      } else {
+        this.set('attemptContainerList', null);
+        this.set('selectedContainerId', '');
+        this.set('containerLogFiles', null);
+        this.set('selectedLogFileName', '');
+        this.set('selectedLogFileContent', '');
+      }
+    },
+
+    showLogFilesForContainerId(containerId) {
+      this.set('selectedContainerId', '');
+      this.set('containerLogFiles', null);
+      this.set('selectedLogFileName', '');
+      this.set('selectedLogFileContent', '');
+      if (containerId) {
+        this.set('_isLoadingBottomPanel', true);
+        this.set('selectedContainerId', containerId);
+        this.fetchLogFilesForContainerId(containerId).then((hash) => {
+          if (hash.logs.get('length') > 0) {
+            this.set('containerLogFiles', hash.logs);
+          } else {
+            this.set('containerLogFiles', null);
+          }
+        }).finally(() => {
+          this.set('_isLoadingBottomPanel', false);
+        });
+      }
+    },
+
+    showContentForLogFile(logFile) {
+      this.set('selectedLogFileName', '');
+      Ember.$("#logContentTextArea1234554321").val('');
+      this.set('showFullLog', false);
+      if (logFile) {
+        this.set('_isLoadingBottomPanel', true);
+        this.set('selectedLogFileName', logFile);
+        this.fetchContentForLogFile(this.get('selectedContainerId'), logFile).then((content) => {
+          this.set('selectedLogFileContent', content.trim());
+        }, () => {
+          this.set('selectedLogFileContent', '');
+        }).always(() => {
+          this.set('_isLoadingBottomPanel', false);
+        });
+      } else {
+        this.set('selectedLogFileContent', '');
+      }
+    },
+
+    findNextTextInLogContent() {
+      let searchInputElem = document.getElementById('logSeachInput98765');
+      this.send('searchTextInLogContent', searchInputElem.value);
+    },
+
+    searchTextInLogContent(searchText) {
+      Ember.$('body').scrollTop(278);
+      let textAreaElem = document.getElementById('logContentTextArea1234554321');
+      let logContent = textAreaElem.innerText;
+      let startIndex = this.searchTextStartIndex || 0;
+      if (startIndex === -1) {
+        startIndex = this.searchTextStartIndex = 0;
+      }
+      if (this.prevSearchText !== searchText) {
+        startIndex = this.searchTextStartIndex = 0;
+      }
+      if (searchText && searchText.trim()) {
+        searchText = searchText.trim();
+        this.prevSearchText = searchText;
+        if (startIndex === 0) {
+          startIndex = logContent.indexOf(searchText, 0);
+        }
+        let endIndex = startIndex + searchText.length;
+        if (document.createRange && window.getSelection) {
+          let range = document.createRange();
+          range.selectNodeContents(textAreaElem);
+          range.setStart(textAreaElem.childNodes.item(0), startIndex);
+          range.setEnd(textAreaElem.childNodes.item(0), endIndex);
+          let selection = window.getSelection();
+          selection.removeAllRanges();
+          selection.addRange(range);
+        }
+        this.searchTextStartIndex = logContent.indexOf(searchText, endIndex + 1);
+      } else {
+        this.searchTextStartIndex = 0;
+      }
+    },
+
+    showFullLogFileContent() {
+      this.set('showFullLog', true);
+      this.notifyPropertyChange('selectedLogFileContent');
+    }
+  },
+
+  attemptList: Ember.computed('model.attempts', function() {
+    let attempts = this.get('model.attempts');
+    let list = null;
+    if (attempts && attempts.get('length') && attempts.get('content')) {
+      list = [].concat(attempts.get('content'));
+    }
+    return list;
+  }),
+
+  fetchContainersForAttemptId(attemptId) {
+    return Ember.RSVP.hash({
+      rmContainers: this.store.query('yarn-container', {
+        app_attempt_id: attemptId
+      }).catch(function() {
+        return Ember.A();
+      }),
+      tsContainers: this.store.query('yarn-timeline-container', {
+        app_attempt_id: attemptId
+      }).catch(function() {
+        return Ember.A();
+      })
+    });
+  },
+
+  fetchLogFilesForContainerId(containerId) {
+    return Ember.RSVP.hash({
+      logs: this.store.query('yarn-log', {
+        containerId: containerId
+      }).catch(function() {
+        return Ember.A();
+      })
+    });
+  },
+
+  fetchContentForLogFile(containerId, logFile) {
+    let logAdapter = this.store.adapterFor('yarn-log');
+    return logAdapter.fetchLogFileContent(containerId, logFile);
+  },
+
+  resetAfterRefresh() {
+    this.set('selectedAttemptId', '');
+    this.set('attemptContainerList', null);
+    this.set('selectedContainerId', '');
+    this.set('selectedLogFileName', '');
+    this.set('containerLogFiles', null);
+    this.set('selectedLogFileContent', '');
+  },
+
+  showFullLog: false,
+
+  showLastFewLinesOfLogContent: Ember.computed('selectedLogFileContent', function() {
+    let content = this.get('selectedLogFileContent');
+    let lines = content.split('\n');
+    if (this.get('showFullLog') || lines.length < 10) {
+      return content;
+    }
+    return lines.slice(lines.length - 10).join('\n');
+  })
+
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-log.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-log.js
new file mode 100644
index 0000000..f022bc7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-log.js
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import DS from 'ember-data';
+
+export default DS.Model.extend({
+  fileName: DS.attr('string'),
+  fileSize: DS.attr('string'),
+  lastModifiedTime: DS.attr('string'),
+  containerId: DS.attr('string'),
+  nodeId: DS.attr('string')
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/router.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/router.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/router.js
index dea2733..bd7af21 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/router.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/router.js
@@ -60,6 +60,7 @@ Router.map(function() {
     this.route('components');
     this.route('charts');
     this.route('configs');
+    this.route('logs');
   });
   this.route('yarn-component-instances', function() {
     this.route('info', {path: '/:component_name/info'});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-app-attempt.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-app-attempt.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-app-attempt.js
index b561bf6..829cfe9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-app-attempt.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-app-attempt.js
@@ -26,11 +26,13 @@ export default AbstractRoute.extend(AppAttemptMixin, {
       attempt: this.fetchAttemptInfoFromRMorATS(param.app_attempt_id, this.store),
       rmContainers: this.store.query('yarn-container', {
         app_attempt_id: param.app_attempt_id
+      }).catch(function() {
+        return Ember.A();
       }),
       tsContainers: this.store.query('yarn-timeline-container', {
         app_attempt_id: param.app_attempt_id
       }).catch(function() {
-        return [];
+        return Ember.A();
       })
     });
   },

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-app/attempts.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-app/attempts.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-app/attempts.js
index 339a10d..451ecc2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-app/attempts.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-app/attempts.js
@@ -29,7 +29,9 @@ export default AbstractRoute.extend(AppAttemptMixin, {
     return Ember.RSVP.hash({
       appId: app_id,
       serviceName: service,
-      attempts: this.fetchAttemptListFromRMorATS(app_id, this.store)
+      attempts: this.fetchAttemptListFromRMorATS(app_id, this.store).catch(function() {
+        return Ember.A();
+      })
     });
   },
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-app/logs.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-app/logs.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-app/logs.js
new file mode 100644
index 0000000..66eb662
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-app/logs.js
@@ -0,0 +1,47 @@
+/**
+ * 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';
+import AbstractRoute from '../abstract';
+import AppAttemptMixin from 'yarn-ui/mixins/app-attempt';
+
+export default AbstractRoute.extend(AppAttemptMixin, {
+  model(param, transition) {
+    const { app_id } = this.paramsFor('yarn-app');
+    const { service } = param;
+    transition.send('updateBreadcrumbs', app_id, service, [{text: 'Logs'}]);
+    return Ember.RSVP.hash({
+      appId: app_id,
+      serviceName: service,
+      attempts: this.fetchAttemptListFromRMorATS(app_id, this.store).catch(function() {
+        return [];
+      })
+    });
+  },
+
+  unloadAll() {
+    this.store.unloadAll('yarn-app-attempt');
+    this.store.unloadAll('yarn-timeline-appattempt');
+    this.store.unloadAll('yarn-container');
+    this.store.unloadAll('yarn-timeline-container');
+    this.store.unloadAll('yarn-log');
+    if (this.controller) {
+      this.controller.resetAfterRefresh();
+    }
+  }
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-log.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-log.js
new file mode 100644
index 0000000..93174d0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-log.js
@@ -0,0 +1,48 @@
+/**
+ * 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 DS from 'ember-data';
+
+export default DS.JSONAPISerializer.extend({
+  internalNormalizeSingleResponse(store, primaryModelClass, payload, containerId, nodeId) {
+    var fixedPayload = {
+      id: "yarn_log_" + payload.fileName + "_" + Date.now(),
+      type: primaryModelClass.modelName,
+      attributes: {
+        fileName: payload.fileName,
+        fileSize: payload.fileSize,
+        lastModifiedTime: payload.lastModifiedTime,
+        containerId: containerId,
+        nodeId: nodeId
+      }
+    };
+    return fixedPayload;
+  },
+
+  normalizeArrayResponse(store, primaryModelClass, payload/*, id, requestType*/) {
+    var normalizedArrayResponse = {
+      data: []
+    };
+    if (payload && payload.containerLogsInfo && payload.containerLogsInfo.containerLogInfo) {
+      normalizedArrayResponse.data = payload.containerLogsInfo.containerLogInfo.map((paylog) => {
+        return this.internalNormalizeSingleResponse(store, primaryModelClass, paylog,
+          payload.containerLogsInfo.containerId, payload.containerLogsInfo.nodeId);
+      });
+    }
+    return normalizedArrayResponse;
+  }
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/collapsible-panel.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/collapsible-panel.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/collapsible-panel.hbs
new file mode 100644
index 0000000..aab80b1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/collapsible-panel.hbs
@@ -0,0 +1,21 @@
+{{!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+}}
+
+<a class="pull-right toggle_switch" href="#" {{action "togglePanelCollapse"}}>
+  <span class="glyphicon glyphicon-chevron-up"></span>
+</a>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/timeline-view.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/timeline-view.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/timeline-view.hbs
index 9e32e47..13348b2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/timeline-view.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/timeline-view.hbs
@@ -25,49 +25,41 @@
         Containers
       {{/if}}
     </div>
-    {{#if isDataEmpty}}
-      <ul class="nav nav-tabs" role="tablist">
-        <li class="active">
-          <a href="#graphViewTab" role="tab" data-toggle="tab">Graph View</a>
-        </li>
-        <li class="">
-          <a href="#gridViewTab" role="tab" data-toggle="tab">Grid View</a>
-        </li>
-      </ul>
-      <div class="panel-body">
-        <div class="tab-content">
-          <div role="tabpanel" class="tab-pane active" id="graphViewTab">
-            <br/><br/>
-            <div class="col-md-7 container-fluid" id={{parent-id}}></div>
-            <!-- diag info -->
-            <div class="col-md-5 container-fluid">
-              <div class="panel panel-default add-ellipsis attempt-info-panel">
-                <div class="panel-heading">
-                  {{#if selected.link}}
-                    {{#link-to selected.linkname selected.id (query-params service=serviceName)}}{{selected.id}}{{/link-to}}
-                  {{else}}
-                    {{selected.id}}
-                  {{/if}}
-                </div>
-                {{#if attemptModel}}
-                  {{app-attempt-table attempt=selected}}
+    <ul class="nav nav-tabs" role="tablist">
+      <li class="active">
+        <a href="#graphViewTab" role="tab" data-toggle="tab">Graph View</a>
+      </li>
+      <li class="">
+        <a href="#gridViewTab" role="tab" data-toggle="tab">Grid View</a>
+      </li>
+    </ul>
+    <div class="panel-body">
+      <div class="tab-content">
+        <div role="tabpanel" class="tab-pane active" id="graphViewTab">
+          <br/><br/>
+          <div class="col-md-7 container-fluid" id={{parent-id}}></div>
+          <!-- diag info -->
+          <div class="col-md-5 container-fluid">
+            <div class="panel panel-default add-ellipsis attempt-info-panel">
+              <div class="panel-heading">
+                {{#if selected.link}}
+                  {{#link-to selected.linkname selected.id (query-params service=serviceName)}}{{selected.id}}{{/link-to}}
                 {{else}}
-                  {{container-table container=selected}}
+                  {{selected.id}}
                 {{/if}}
               </div>
+              {{#if attemptModel}}
+                {{app-attempt-table attempt=selected}}
+              {{else}}
+                {{container-table container=selected}}
+              {{/if}}
             </div>
           </div>
-          <div role="tabpanel" class="tab-pane" id="gridViewTab">
-            {{em-table columns=gridColumns rows=gridRows}}
-          </div>
+        </div>
+        <div role="tabpanel" class="tab-pane" id="gridViewTab">
+          {{em-table columns=gridColumns rows=gridRows}}
         </div>
       </div>
-    {{else}}
-      <div class="panel-body">
-        <h4 class="text-center">No data available!</h4>
-      </div>
-    {{/if}}
+    </div>
   </div>
 </div>
-
-{{outlet}}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app-attempt.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app-attempt.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app-attempt.hbs
index 7579bdc..2e2f6d0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app-attempt.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app-attempt.hbs
@@ -48,7 +48,26 @@
     </div>
     <div class="row">
       {{#if (or model.rmContainers model.tsContainers)}}
-        {{timeline-view parent-id="containers-timeline-div" my-id="timeline-view" height="400" rmModel=model.rmContainers tsModel=model.tsContainers label="shortAppAttemptId" attemptModel=false}}
+        {{timeline-view
+          parent-id="containers-timeline-div"
+          my-id="timeline-view"
+          height="400"
+          rmModel=model.rmContainers
+          tsModel=model.tsContainers
+          label="shortAppAttemptId"
+          attemptModel=false
+        }}
+      {{else}}
+        <div class="col-md-12 container-fluid">
+          <div class="panel panel-default">
+            <div class="panel-heading">
+               Containers
+            </div>
+            <div class="panel-body">
+              <h4 class="text-center">No data available!</h4>
+            </div>
+          </div>
+        </div>
       {{/if}}
     </div>
   </div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/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 fc40837..ca93428 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
@@ -125,6 +125,9 @@
             {{#link-to 'yarn-app.info' tagName="li" class=(if (eq target.currentPath 'yarn-app.info') "active")}}
               {{#link-to 'yarn-app.info' appId (query-params service=model.serviceName)}}Diagnostics{{/link-to}}
             {{/link-to}}
+            {{#link-to 'yarn-app.logs' tagName="li" class=(if (eq target.currentPath 'yarn-app.logs') "active")}}
+              {{#link-to 'yarn-app.logs' appId (query-params service=serviceName)}}Logs{{/link-to}}
+            {{/link-to}}
           </ul>
         </ul>
       </div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app/attempts.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app/attempts.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app/attempts.hbs
index 37ed079..5857d89 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app/attempts.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app/attempts.hbs
@@ -17,6 +17,7 @@
 }}
 
 <div>
+{{#if model.attempts}}
   {{timeline-view
     parent-id="attempt-timeline-div"
     my-id="timeline-view"
@@ -26,4 +27,16 @@
     attemptModel=true
     serviceName=model.serviceName
   }}
+{{else}}
+  <div class="col-md-12 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+         Application Attempts
+      </div>
+      <div class="panel-body">
+        <h4 class="text-center">No data available!</h4>
+      </div>
+    </div>
+  </div>
+{{/if}}
 </div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app/logs.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app/logs.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app/logs.hbs
new file mode 100644
index 0000000..34c6ee1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-app/logs.hbs
@@ -0,0 +1,132 @@
+{{!
+ * 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.
+}}
+
+<div class="row">
+  <div class="col-md-12">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Logs {{collapsible-panel targetId="logFilesCollapsablePanel"}}
+      </div>
+      <div class="panel-body" id="logFilesCollapsablePanel">
+        {{#if _isLoadingTopPanel}}
+          <div class="text-center" style="z-index: 100; position: absolute; left: 46%;">
+            <img src="assets/images/spinner.gif" alt="Loading...">
+          </div>
+        {{/if}}
+        {{#if attemptList}}
+          <div class="row">
+            <div class="col-md-6">
+              <label>Choose attempt to fetch containers</label>
+              <select class="form-control" onchange={{action "showContainersForAttemptId" value="target.value"}} style="max-width:350px;">
+                <option value="" selected={{eq selectedAttemptId ''}}>None</option>
+                {{#each attemptList as |attempt|}}
+                  <option value="{{attempt.id}}" selected={{eq selectedAttemptId attempt.id}}>{{attempt.id}}</option>
+                {{/each}}
+              </select>
+            </div>
+            {{#if attemptContainerList}}
+              <div class="col-md-6">
+                <label>Choose container to fetch logs</label>
+                <select class="form-control" onchange={{action "showLogFilesForContainerId" value="target.value"}} style="max-width:350px">
+                  <option value="" selected={{eq selectedContainerId ''}}>None</option>
+                  {{#each attemptContainerList as |container|}}
+                    <option value="{{container.id}}" selected={{eq selectedContainerId container.id}}>{{container.id}}</option>
+                  {{/each}}
+                </select>
+              </div>
+            {{else}}
+              {{#if (and selectedAttemptId (not _isLoadingTopPanel))}}
+                <div class="col-md-4">
+                  <h4 class="text-center" style="margin-top:25px;">No container data available!</h4>
+                </div>
+              {{/if}}
+            {{/if}}
+          </div>
+        {{else}}
+          <div class="col-md-12">
+            <h4 class="text-center">No data available!</h4>
+          </div>
+        {{/if}}
+      </div>
+    </div>
+  </div>
+</div>
+{{#if selectedContainerId}}
+  <div class="row">
+    <div class="col-md-12">
+      <div class="panel panel-default" style="min-height:150px;">
+        <div class="panel-heading">
+          Log: {{selectedLogFileName}} [ {{selectedContainerId}} &amp; {{selectedAttemptId}} ]
+          {{collapsible-panel targetId="logContentCollapsablePanel"}}
+        </div>
+        <div class="panel-body" id="logContentCollapsablePanel">
+          {{#if _isLoadingBottomPanel}}
+            <div class="text-center" style="z-index: 100; position: absolute; left: 46%;">
+              <img src="assets/images/spinner.gif" alt="Loading...">
+            </div>
+          {{/if}}
+          {{#if containerLogFiles}}
+            <div class="row">
+              <div class="col-md-6" style="margin-bottom:20px;">
+                <label>Choose log for {{selectedContainerId}}</label>
+                <select class="form-control" onchange={{action "showContentForLogFile" value="target.value"}} style="max-width:350px">
+                  <option value="" selected={{eq selectedLogFileName ''}}>None</option>
+                  {{#each containerLogFiles as |file|}}
+                    <option value="{{file.fileName}}" selected={{eq selectedLogFileName file.fileName}}>{{file.fileName}} - {{file.fileSize}} bytes</option>
+                  {{/each}}
+                </select>
+              </div>
+              {{#if selectedLogFileName}}
+                <div class="col-md-6">
+                  <label>Find in log</label>
+                  <div class="input-group" style="max-width:350px;">
+                    {{input
+                      class="form-control"
+                      value=""
+                      enter="searchTextInLogContent"
+                      id="logSeachInput98765"
+                    }}
+                    <span class="input-group-btn">
+                      <button class="btn btn-default" {{action "findNextTextInLogContent"}}>Find</button>
+                    </span>
+                  </div>
+                </div>
+              {{/if}}
+            </div>
+            <div class="row">
+              {{#if selectedLogFileContent}}
+                {{#unless showFullLog}}
+                  <div class="col-md-12">
+                    <strong>Showing last 10 lines of log. Click <a href="#" {{action "showFullLogFileContent"}}>here</a> for full log.</strong>
+                  </div>
+                {{/unless}}
+                <pre id="logContentTextArea1234554321" class="log-content-area">{{showLastFewLinesOfLogContent}}</pre>
+              {{/if}}
+            </div>
+          {{else}}
+            {{#unless _isLoadingBottomPanel}}
+              <div class="col-md-12" style="margin-top:20px;">
+                <h4 class="text-center">No log data available!</h4>
+              </div>
+            {{/unless}}
+          {{/if}}
+        </div>
+      </div>
+    </div>
+  </div>
+{{/if}}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower-shrinkwrap.json
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower-shrinkwrap.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower-shrinkwrap.json
index b0f3aa3..ffe3805 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower-shrinkwrap.json
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower-shrinkwrap.json
@@ -1,6 +1,6 @@
 {
   "https://github.com/DataTables/DataTables.git": {
-    "1.10.15": "1.10.15"
+    "1.10.15": "84d24d41477bac7657c9ddfa1d86245f9affc879"
   },
   "https://github.com/components/ember-data.git": {
     "2.1.0": "d8b4d3092f67afe22d9d374c40d719d557915fa3"
@@ -27,10 +27,10 @@
     "0.1.7": "7bb21488563bd1bba23e903a812bf5815beddd1a"
   },
   "https://github.com/fgnass/spin.js.git": {
-    "2.3.2": "2.3.2"
+    "2.3.2": "5a9ea41651105302fe76d15cb294070c5863e759"
   },
   "https://github.com/ivaynberg/select2.git": {
-    "4.0.0": "4.0.0"
+    "4.0.0": "80eb44bec78568b152dc5547269c164cb7b5363c"
   },
   "https://github.com/jquery/jquery-dist.git": {
     "2.1.4": "7751e69b615c6eca6f783a81e292a55725af6b85"
@@ -39,20 +39,20 @@
     "1.19.0": "467e7e34652ad7d5883ce9c568461cf8c5e172a8"
   },
   "https://github.com/mbostock-bower/d3-bower.git": {
-    "3.5.17": "3.5.17"
+    "3.5.17": "abe0262a205c9f3755c3a757de4dfd1d49f34b24"
   },
   "https://github.com/moment/moment-timezone.git": {
     "0.5.0": "74a2e9378ecf4a31a168f3049f086565c8d66814"
   },
   "https://github.com/moment/moment.git": {
-    "2.10.6": "2.10.6",
+    "2.10.6": "446ce77eb08c5c862d7b0b11ef1d2e884d12e3d7",
     "2.12.0": "d3d7488b4d60632854181cb0a9af325d57fb3d51"
   },
   "https://github.com/rwjblue/ember-qunit-builds.git": {
     "0.4.16": "142c4066a5458bef9dfcb92b70152b9c01d79188"
   },
   "https://github.com/sreenaths/more-js.git": {
-    "0.8.2": "0.8.2"
+    "0.8.2": "015ab7c7dad2dc9edc0049b8b65aeb020fd20c51"
   },
   "https://github.com/sreenaths/snippet-ss.git": {
     "1.11.0": "c1abc566f4e001b7f1939b6dbdd911eadc969cf9"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/integration/components/collapsible-panel-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/integration/components/collapsible-panel-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/integration/components/collapsible-panel-test.js
new file mode 100644
index 0000000..b80eb07
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/integration/components/collapsible-panel-test.js
@@ -0,0 +1,43 @@
+/**
+ * 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 { moduleForComponent, test } from 'ember-qunit';
+import hbs from 'htmlbars-inline-precompile';
+
+moduleForComponent('collapsible-panel', 'Integration | Component | collapsible panel', {
+  integration: true
+});
+
+test('it renders', function(assert) {
+
+  // Set any properties with this.set('myProperty', 'value');
+  // Handle any actions with this.on('myAction', function(val) { ... });" + EOL + EOL +
+
+  this.render(hbs`{{collapsible-panel}}`);
+
+  assert.equal(this.$().text().trim(), '');
+
+  // Template block usage:" + EOL +
+  this.render(hbs`
+    {{#collapsible-panel}}
+      template block text
+    {{/collapsible-panel}}
+  `);
+
+  assert.equal(this.$().text().trim(), 'template block text');
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/adapters/yarn-log-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/adapters/yarn-log-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/adapters/yarn-log-test.js
new file mode 100644
index 0000000..04664cc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/adapters/yarn-log-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('adapter:yarn-log', 'Unit | Adapter | yarn log', {
+  // Specify the other units that are required for this test.
+  // needs: ['serializer:foo']
+});
+
+// Replace this with your real tests.
+test('it exists', function(assert) {
+  let adapter = this.subject();
+  assert.ok(adapter);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-app/logs-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-app/logs-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-app/logs-test.js
new file mode 100644
index 0000000..87c8dde
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/controllers/yarn-app/logs-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-app/logs', 'Unit | Controller | yarn app/logs', {
+  // 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);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/models/yarn-log-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/models/yarn-log-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/models/yarn-log-test.js
new file mode 100644
index 0000000..6c331b4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/models/yarn-log-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 { moduleForModel, test } from 'ember-qunit';
+
+moduleForModel('yarn-log', 'Unit | Model | yarn log', {
+  // Specify the other units that are required for this test.
+  needs: []
+});
+
+test('it exists', function(assert) {
+  let model = this.subject();
+  // let store = this.store();
+  assert.ok(!!model);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-app/logs-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-app/logs-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-app/logs-test.js
new file mode 100644
index 0000000..e1bc64e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/routes/yarn-app/logs-test.js
@@ -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.
+ */
+
+import { moduleFor, test } from 'ember-qunit';
+
+moduleFor('route:yarn-app/logs', 'Unit | Route | yarn app/logs', {
+  // Specify the other units that are required for this test.
+  // needs: ['controller:foo']
+});
+
+test('it exists', function(assert) {
+  let route = this.subject();
+  assert.ok(route);
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99ccca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/serializers/yarn-log-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/serializers/yarn-log-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/serializers/yarn-log-test.js
new file mode 100644
index 0000000..a84e689
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/unit/serializers/yarn-log-test.js
@@ -0,0 +1,33 @@
+/**
+ * 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 { moduleForModel, test } from 'ember-qunit';
+
+moduleForModel('yarn-log', 'Unit | Serializer | yarn log', {
+  // Specify the other units that are required for this test.
+  needs: ['serializer:yarn-log']
+});
+
+// Replace this with your real tests.
+test('it serializes records', function(assert) {
+  let record = this.subject();
+
+  let serializedRecord = record.serialize();
+
+  assert.ok(serializedRecord);
+});


---------------------------------------------------------------------
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-6851. Capacity Scheduler: document configs for controlling # containers allowed to be allocated per node heartbeat.

Posted by ae...@apache.org.
YARN-6851. Capacity Scheduler: document configs for controlling # containers allowed to be allocated per node heartbeat.


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

Branch: refs/heads/HDFS-7240
Commit: 333ef303ff0caf9adfd378652a8f966377901768
Parents: 53509f2
Author: Wei Yan <we...@apache.org>
Authored: Wed Nov 29 14:37:57 2017 -0800
Committer: Wei Yan <we...@apache.org>
Committed: Wed Nov 29 14:38:07 2017 -0800

----------------------------------------------------------------------
 .../src/site/markdown/CapacityScheduler.md                | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/333ef303/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 4754c94..0449e88 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
@@ -282,6 +282,16 @@ The `ReservationSystem` is integrated with the `CapacityScheduler` queue hierach
 |:---- |:---- |
 | `yarn.scheduler.capacity.node-locality-delay` | Number of missed scheduling opportunities after which the CapacityScheduler attempts to schedule rack-local containers. Typically, this should be set to number of nodes in the cluster. By default is setting approximately number of nodes in one rack which is 40. Positive integer value is expected. |
 
+  * Container Allocation per NodeManager Heartbeat
+
+  The `CapacityScheduler` supports the following parameters to control how many containers can be allocated in each NodeManager heartbeat.
+
+| Property | Description |
+|:---- |:---- |
+| `yarn.scheduler.capacity.per-node-heartbeat.multiple-assignments-enabled` | Whether to allow multiple container assignments in one NodeManager heartbeat. Defaults to true. |
+| `yarn.scheduler.capacity.per-node-heartbeat.maximum-container-assignments` | If `multiple-assignments-enabled` is true, the maximum amount of containers that can be assigned in one NodeManager heartbeat. Defaults to -1, which sets no limit. |
+| `yarn.scheduler.capacity.per-node-heartbeat.maximum-offswitch-assignments` | If `multiple-assignments-enabled` is true, the maximum amount of off-switch containers that can be assigned in one NodeManager heartbeat. Defaults to 1, which represents only one off-switch allocation allowed in one heartbeat. |
+
 ###Reviewing the configuration of the CapacityScheduler
 
   Once the installation and configuration is completed, you can review it after starting the YARN cluster from the web-ui.


---------------------------------------------------------------------
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-12877. Add open(PathHandle) with default buffersize

Posted by ae...@apache.org.
HDFS-12877. Add open(PathHandle) with default buffersize


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

Branch: refs/heads/HDFS-7240
Commit: 0780fdb1ebdddd19744fbbca7fb05f8fe4bf4d28
Parents: a409425
Author: Chris Douglas <cd...@apache.org>
Authored: Thu Nov 30 15:13:16 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Thu Nov 30 15:13:16 2017 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/fs/FileSystem.java   | 15 +++++++++++++++
 .../org/apache/hadoop/fs/TestFilterFileSystem.java   |  1 +
 .../java/org/apache/hadoop/fs/TestHarFileSystem.java |  1 +
 3 files changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0780fdb1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index be0ec87..a364921 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -957,6 +957,21 @@ public abstract class FileSystem extends Configured implements Closeable {
    * resource directly and verify that the resource referenced
    * satisfies constraints specified at its construciton.
    * @param fd PathHandle object returned by the FS authority.
+   * @throws IOException IO failure
+   * @throws UnsupportedOperationException If {@link #open(PathHandle, int)}
+   *                                       not overridden by subclass
+   */
+  public FSDataInputStream open(PathHandle fd) throws IOException {
+    return open(fd, getConf().getInt(IO_FILE_BUFFER_SIZE_KEY,
+        IO_FILE_BUFFER_SIZE_DEFAULT));
+  }
+
+  /**
+   * Open an FSDataInputStream matching the PathHandle instance. The
+   * implementation may encode metadata in PathHandle to address the
+   * resource directly and verify that the resource referenced
+   * satisfies constraints specified at its construciton.
+   * @param fd PathHandle object returned by the FS authority.
    * @param bufferSize the size of the buffer to use
    * @throws IOException IO failure
    * @throws UnsupportedOperationException If not overridden by subclass

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0780fdb1/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
index 4cbb8ab..0e9a612 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
@@ -79,6 +79,7 @@ public class TestFilterFileSystem {
 
     public boolean mkdirs(Path f);
     public FSDataInputStream open(Path f);
+    public FSDataInputStream open(PathHandle f);
     public FSDataOutputStream create(Path f);
     public FSDataOutputStream create(Path f, boolean overwrite);
     public FSDataOutputStream create(Path f, Progressable progress);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0780fdb1/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
index a1aa4de..1b69693 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
@@ -80,6 +80,7 @@ public class TestHarFileSystem {
 
     public boolean mkdirs(Path f);
     public FSDataInputStream open(Path f);
+    public FSDataInputStream open(PathHandle f);
     public FSDataOutputStream create(Path f);
     public FSDataOutputStream create(Path f, boolean overwrite);
     public FSDataOutputStream create(Path f, Progressable progress);


---------------------------------------------------------------------
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: MAPREDUCE-6994. Uploader tool for Distributed Cache Deploy code changes (miklos.szegedi@cloudera.com via rkanter)

Posted by ae...@apache.org.
MAPREDUCE-6994. Uploader tool for Distributed Cache Deploy code changes  (miklos.szegedi@cloudera.com via rkanter)


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

Branch: refs/heads/HDFS-7240
Commit: 3b78607a02f3a81ad730975ecdfa35967413271d
Parents: 21d3627
Author: Robert Kanter <rk...@apache.org>
Authored: Fri Dec 1 12:11:43 2017 -0800
Committer: Robert Kanter <rk...@apache.org>
Committed: Fri Dec 1 12:12:15 2017 -0800

----------------------------------------------------------------------
 hadoop-mapreduce-project/bin/mapred             |   4 +
 .../hadoop-mapreduce-client-uploader/pom.xml    |  67 ++++
 .../hadoop/mapred/uploader/DefaultJars.java     |  46 +++
 .../mapred/uploader/FrameworkUploader.java      | 384 +++++++++++++++++++
 .../mapred/uploader/UploaderException.java      |  36 ++
 .../hadoop/mapred/uploader/package-info.java    |  28 ++
 .../mapred/uploader/TestFrameworkUploader.java  | 315 +++++++++++++++
 .../hadoop-mapreduce-client/pom.xml             |   1 +
 8 files changed, 881 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b78607a/hadoop-mapreduce-project/bin/mapred
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/bin/mapred b/hadoop-mapreduce-project/bin/mapred
index f66f563..ce9ce21 100755
--- a/hadoop-mapreduce-project/bin/mapred
+++ b/hadoop-mapreduce-project/bin/mapred
@@ -32,6 +32,7 @@ function hadoop_usage
   hadoop_add_subcommand "pipes" client "run a Pipes job"
   hadoop_add_subcommand "queue" client "get information regarding JobQueues"
   hadoop_add_subcommand "sampler" client "sampler"
+  hadoop_add_subcommand "frameworkuploader" admin "mapreduce framework upload"
   hadoop_add_subcommand "version" client "print the version"
   hadoop_generate_usage "${HADOOP_SHELL_EXECNAME}" true
 }
@@ -92,6 +93,9 @@ function mapredcmd_case
     sampler)
       HADOOP_CLASSNAME=org.apache.hadoop.mapred.lib.InputSampler
     ;;
+    frameworkuploader)
+      HADOOP_CLASSNAME=org.apache.hadoop.mapred.uploader.FrameworkUploader
+    ;;
     version)
       HADOOP_CLASSNAME=org.apache.hadoop.util.VersionInfo
     ;;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b78607a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/pom.xml
new file mode 100644
index 0000000..a721404
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/pom.xml
@@ -0,0 +1,67 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>hadoop-mapreduce-client</artifactId>
+        <groupId>org.apache.hadoop</groupId>
+        <version>3.1.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>hadoop-mapreduce-client-uploader</artifactId>
+    <version>3.1.0-SNAPSHOT</version>
+    <name>Apache Hadoop MapReduce Uploader</name>
+
+    <dependencies>
+        <dependency>
+            <groupId>commons-cli</groupId>
+            <artifactId>commons-cli</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-compress</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs-client</artifactId>
+        </dependency>
+    </dependencies>
+    <properties>
+        <!-- Needed for generating FindBugs warnings using parent pom -->
+        <mr.basedir>${project.parent.basedir}/../</mr.basedir>
+    </properties>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <configuration>
+                    <archive>
+                        <manifest>
+                            <mainClass>org.apache.hadoop.mapred.uploader.FrameworkUploader</mainClass>
+                        </manifest>
+                    </archive>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b78607a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/main/java/org/apache/hadoop/mapred/uploader/DefaultJars.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/main/java/org/apache/hadoop/mapred/uploader/DefaultJars.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/main/java/org/apache/hadoop/mapred/uploader/DefaultJars.java
new file mode 100644
index 0000000..49ee64f
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/main/java/org/apache/hadoop/mapred/uploader/DefaultJars.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.mapred.uploader;
+
+/**
+ * Default white list and black list implementations.
+ */
+final class DefaultJars {
+  static final String DEFAULT_EXCLUDED_MR_JARS =
+      ".*hadoop-yarn-server-applicationhistoryservice.*\\.jar," +
+          ".*hadoop-yarn-server-nodemanager.*\\.jar," +
+          ".*hadoop-yarn-server-resourcemanager.*\\.jar," +
+          ".*hadoop-yarn-server-router.*\\.jar," +
+          ".*hadoop-yarn-server-sharedcachemanager.*\\.jar," +
+          ".*hadoop-yarn-server-timeline-pluginstorage.*\\.jar," +
+          ".*hadoop-yarn-server-timelineservice.*\\.jar," +
+          ".*hadoop-yarn-server-timelineservice-hbase.*\\.jar,";
+
+  static final String DEFAULT_MR_JARS =
+      "$HADOOP_HOME/share/hadoop/common/.*\\.jar," +
+          "$HADOOP_HOME/share/hadoop/common/lib/.*\\.jar," +
+          "$HADOOP_HOME/share/hadoop/hdfs/.*\\.jar," +
+          "$HADOOP_HOME/share/hadoop/hdfs/lib/.*\\.jar," +
+          "$HADOOP_HOME/share/hadoop/mapreduce/.*\\.jar," +
+          "$HADOOP_HOME/share/hadoop/mapreduce/lib/.*\\.jar," +
+          "$HADOOP_HOME/share/hadoop/yarn/.*\\.jar," +
+          "$HADOOP_HOME/share/hadoop/yarn/lib/.*\\.jar,";
+
+  private DefaultJars() {}
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b78607a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/main/java/org/apache/hadoop/mapred/uploader/FrameworkUploader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/main/java/org/apache/hadoop/mapred/uploader/FrameworkUploader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/main/java/org/apache/hadoop/mapred/uploader/FrameworkUploader.java
new file mode 100644
index 0000000..d1cd740
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/main/java/org/apache/hadoop/mapred/uploader/FrameworkUploader.java
@@ -0,0 +1,384 @@
+/**
+ * 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.mapred.uploader;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.compress.archivers.ArchiveEntry;
+import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.GenericOptionsParser;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.zip.GZIPOutputStream;
+
+/**
+ * Upload a MapReduce framework tarball to HDFS.
+ * Usage:
+ * sudo -u mapred mapred frameworkuploader -fs hdfs://`hostname`:8020 -target
+ * /tmp/upload.tar.gz#mr-framework
+*/
+public class FrameworkUploader implements Runnable {
+  private static final Pattern VAR_SUBBER =
+      Pattern.compile(Shell.getEnvironmentVariableRegex());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FrameworkUploader.class);
+
+  @VisibleForTesting
+  String input = null;
+  @VisibleForTesting
+  String whitelist = null;
+  @VisibleForTesting
+  String blacklist = null;
+  @VisibleForTesting
+  String target = null;
+  @VisibleForTesting
+  short replication = 10;
+
+  @VisibleForTesting
+  Set<String> filteredInputFiles = new HashSet<>();
+  @VisibleForTesting
+  List<Pattern> whitelistedFiles = new LinkedList<>();
+  @VisibleForTesting
+  List<Pattern> blacklistedFiles = new LinkedList<>();
+
+  @VisibleForTesting
+  OutputStream targetStream = null;
+  private Path targetPath = null;
+  private String alias = null;
+
+  private void printHelp(Options options) {
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp("mapred frameworkuploader", options);
+  }
+
+  public void run() {
+    try {
+      collectPackages();
+      buildPackage();
+      LOG.info("Uploaded " + target);
+      System.out.println("Suggested mapreduce.application.framework.path " +
+          target);
+      LOG.info(
+          "Suggested mapreduce.application.classpath $PWD/" + alias + "/*");
+      System.out.println("Suggested classpath $PWD/" + alias + "/*");
+    } catch (UploaderException|IOException e) {
+      LOG.error("Error in execution " + e.getMessage());
+      e.printStackTrace();
+    }
+  }
+
+  @VisibleForTesting
+  void collectPackages() throws UploaderException {
+    parseLists();
+    String[] list = StringUtils.split(input, File.pathSeparatorChar);
+    for (String item : list) {
+      LOG.info("Original source " + item);
+      String expanded = expandEnvironmentVariables(item, System.getenv());
+      LOG.info("Expanded source " + expanded);
+      if (expanded.endsWith("*")) {
+        File path = new File(expanded.substring(0, expanded.length() - 1));
+        if (path.isDirectory()) {
+          File[] files = path.listFiles();
+          if (files != null) {
+            for (File jar : files) {
+              if (!jar.isDirectory()) {
+                addJar(jar);
+              } else {
+                LOG.info("Ignored " + jar + " because it is a directory");
+              }
+            }
+          } else {
+            LOG.warn("Could not list directory " + path);
+          }
+        } else {
+          LOG.warn("Ignored " + expanded + ". It is not a directory");
+        }
+      } else if (expanded.endsWith(".jar")) {
+        File jarFile = new File(expanded);
+        addJar(jarFile);
+      } else if (!expanded.isEmpty()) {
+        LOG.warn("Ignored " + expanded + " only jars are supported");
+      }
+    }
+  }
+
+  private void beginUpload() throws IOException, UploaderException {
+    if (targetStream == null) {
+      validateTargetPath();
+      int lastIndex = target.indexOf('#');
+      targetPath =
+          new Path(
+              target.substring(
+                  0, lastIndex == -1 ? target.length() : lastIndex));
+      alias = lastIndex != -1 ?
+          target.substring(lastIndex + 1) :
+          targetPath.getName();
+      LOG.info("Target " + targetPath);
+      FileSystem fileSystem = targetPath.getFileSystem(new Configuration());
+      targetStream = fileSystem.create(targetPath, true);
+    }
+  }
+
+  @VisibleForTesting
+  void buildPackage() throws IOException, UploaderException {
+    beginUpload();
+    LOG.info("Compressing tarball");
+    try (TarArchiveOutputStream out = new TarArchiveOutputStream(
+        new GZIPOutputStream(targetStream))) {
+      for (String fullPath : filteredInputFiles) {
+        LOG.info("Adding " + fullPath);
+        File file = new File(fullPath);
+        try (FileInputStream inputStream = new FileInputStream(file)) {
+          ArchiveEntry entry = out.createArchiveEntry(file, file.getName());
+          out.putArchiveEntry(entry);
+          IOUtils.copyBytes(inputStream, out, 1024 * 1024);
+          out.closeArchiveEntry();
+        }
+      }
+    } finally {
+      if (targetStream != null) {
+        targetStream.close();
+      }
+    }
+
+    if (targetPath == null) {
+      return;
+    }
+
+    // Set file attributes
+    FileSystem fileSystem = targetPath.getFileSystem(new Configuration());
+    if (fileSystem instanceof DistributedFileSystem) {
+      LOG.info("Disabling Erasure Coding for path: " + targetPath);
+      DistributedFileSystem dfs = (DistributedFileSystem) fileSystem;
+      dfs.setErasureCodingPolicy(targetPath,
+          SystemErasureCodingPolicies.getReplicationPolicy().getName());
+    }
+
+    if (replication > 0) {
+      LOG.info("Set replication to " +
+          replication + " for path: " + targetPath);
+      fileSystem.setReplication(targetPath, replication);
+    }
+  }
+
+  private void parseLists() throws UploaderException {
+    Map<String, String> env = System.getenv();
+    for(Map.Entry<String, String> item : env.entrySet()) {
+      LOG.info("Environment " + item.getKey() + " " + item.getValue());
+    }
+    String[] whiteListItems = StringUtils.split(whitelist);
+    for (String pattern : whiteListItems) {
+      String expandedPattern =
+          expandEnvironmentVariables(pattern, env);
+      Pattern compiledPattern =
+          Pattern.compile("^" + expandedPattern + "$");
+      LOG.info("Whitelisted " + compiledPattern.toString());
+      whitelistedFiles.add(compiledPattern);
+    }
+    String[] blacklistItems = StringUtils.split(blacklist);
+    for (String pattern : blacklistItems) {
+      String expandedPattern =
+          expandEnvironmentVariables(pattern, env);
+      Pattern compiledPattern =
+          Pattern.compile("^" + expandedPattern + "$");
+      LOG.info("Blacklisted " + compiledPattern.toString());
+      blacklistedFiles.add(compiledPattern);
+    }
+  }
+
+  @VisibleForTesting
+  String expandEnvironmentVariables(String innerInput, Map<String, String> env)
+      throws UploaderException {
+    boolean found;
+    do {
+      found = false;
+      Matcher matcher = VAR_SUBBER.matcher(innerInput);
+      StringBuffer stringBuffer = new StringBuffer();
+      while (matcher.find()) {
+        found = true;
+        String var = matcher.group(1);
+        // replace $env with the child's env constructed by tt's
+        String replace = env.get(var);
+        // the env key is not present anywhere .. simply set it
+        if (replace == null) {
+          throw new UploaderException("Environment variable does not exist " +
+              var);
+        }
+        matcher.appendReplacement(
+            stringBuffer, Matcher.quoteReplacement(replace));
+      }
+      matcher.appendTail(stringBuffer);
+      innerInput = stringBuffer.toString();
+    } while (found);
+    return innerInput;
+  }
+
+  private void addJar(File jar) throws UploaderException{
+    boolean found = false;
+    if (!jar.getName().endsWith(".jar")) {
+      LOG.info("Ignored non-jar " + jar.getAbsolutePath());
+    }
+    for (Pattern pattern : whitelistedFiles) {
+      Matcher matcher = pattern.matcher(jar.getAbsolutePath());
+      if (matcher.matches()) {
+        LOG.info("Whitelisted " + jar.getAbsolutePath());
+        found = true;
+        break;
+      }
+    }
+    boolean excluded = false;
+    for (Pattern pattern : blacklistedFiles) {
+      Matcher matcher = pattern.matcher(jar.getAbsolutePath());
+      if (matcher.matches()) {
+        LOG.info("Blacklisted " + jar.getAbsolutePath());
+        excluded = true;
+        break;
+      }
+    }
+    if (found && !excluded) {
+      LOG.info("Whitelisted " + jar.getAbsolutePath());
+      if (!filteredInputFiles.add(jar.getAbsolutePath())) {
+        throw new UploaderException("Duplicate jar" + jar.getAbsolutePath());
+      }
+    }
+    if (!found) {
+      LOG.info("Ignored " + jar.getAbsolutePath() + " because it is missing " +
+          "from the whitelist");
+    } else if (excluded) {
+      LOG.info("Ignored " + jar.getAbsolutePath() + " because it is on " +
+          "the the blacklist");
+    }
+  }
+
+  private void validateTargetPath() throws UploaderException {
+    if (!target.startsWith("hdfs:/") &&
+        !target.startsWith("file:/")) {
+      throw new UploaderException("Target path is not hdfs or local " + target);
+    }
+  }
+
+  @VisibleForTesting
+  boolean parseArguments(String[] args) throws IOException {
+    Options opts = new Options();
+    opts.addOption(OptionBuilder.create("h"));
+    opts.addOption(OptionBuilder.create("help"));
+    opts.addOption(OptionBuilder
+        .withDescription("Input class path")
+        .hasArg().create("input"));
+    opts.addOption(OptionBuilder
+        .withDescription(
+            "Regex specifying the full path of jars to include in the" +
+                " framework tarball. Default is a hardcoded set of jars" +
+                " considered necessary to include")
+        .hasArg().create("whitelist"));
+    opts.addOption(OptionBuilder
+        .withDescription(
+            "Regex specifying the full path of jars to exclude in the" +
+                " framework tarball. Default is a hardcoded set of jars" +
+                " considered unnecessary to include")
+        .hasArg().create("blacklist"));
+    opts.addOption(OptionBuilder
+        .withDescription(
+            "Target file system to upload to." +
+            " Example: hdfs://foo.com:8020")
+        .hasArg().create("fs"));
+    opts.addOption(OptionBuilder
+        .withDescription(
+            "Target file to upload to with a reference name." +
+                " Example: /usr/mr-framework.tar.gz#mr-framework")
+        .hasArg().create("target"));
+    opts.addOption(OptionBuilder
+        .withDescription(
+            "Desired replication count")
+        .hasArg().create("replication"));
+    GenericOptionsParser parser = new GenericOptionsParser(opts, args);
+    if (parser.getCommandLine().hasOption("help") ||
+        parser.getCommandLine().hasOption("h")) {
+      printHelp(opts);
+      return false;
+    }
+    input = parser.getCommandLine().getOptionValue(
+        "input", System.getProperty("java.class.path"));
+    whitelist = parser.getCommandLine().getOptionValue(
+        "whitelist", DefaultJars.DEFAULT_MR_JARS);
+    blacklist = parser.getCommandLine().getOptionValue(
+        "blacklist", DefaultJars.DEFAULT_EXCLUDED_MR_JARS);
+    replication = Short.parseShort(parser.getCommandLine().getOptionValue(
+        "replication", "10"));
+    String fs = parser.getCommandLine()
+        .getOptionValue("fs", null);
+    if (fs == null) {
+      LOG.error("Target file system not specified");
+      printHelp(opts);
+      return false;
+    }
+    String path = parser.getCommandLine().getOptionValue("target",
+        "mr-framework.tar.gz#mr-framework");
+    if (path == null) {
+      LOG.error("Target directory not specified");
+      printHelp(opts);
+      return false;
+    }
+    StringBuilder absolutePath = new StringBuilder(fs);
+    absolutePath = absolutePath.append(path.startsWith("/") ? "" : "/");
+    absolutePath.append(path);
+    target = absolutePath.toString();
+
+    if (parser.getRemainingArgs().length > 0) {
+      LOG.warn("Unexpected parameters");
+      printHelp(opts);
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Tool entry point.
+   * @param args arguments
+   * @throws IOException thrown on configuration errors
+   */
+  public static void main(String[] args) throws IOException {
+    FrameworkUploader uploader = new FrameworkUploader();
+    if(uploader.parseArguments(args)) {
+      uploader.run();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b78607a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/main/java/org/apache/hadoop/mapred/uploader/UploaderException.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/main/java/org/apache/hadoop/mapred/uploader/UploaderException.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/main/java/org/apache/hadoop/mapred/uploader/UploaderException.java
new file mode 100644
index 0000000..73f6454
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/main/java/org/apache/hadoop/mapred/uploader/UploaderException.java
@@ -0,0 +1,36 @@
+/**
+ * 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.mapred.uploader;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Framework uploaded exception type.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+class UploaderException extends Exception {
+
+  private static final long serialVersionUID = 1L;
+
+  UploaderException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b78607a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/main/java/org/apache/hadoop/mapred/uploader/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/main/java/org/apache/hadoop/mapred/uploader/package-info.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/main/java/org/apache/hadoop/mapred/uploader/package-info.java
new file mode 100644
index 0000000..4475e8e
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/main/java/org/apache/hadoop/mapred/uploader/package-info.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Package org.apache.hadoop.mapred.uploader contains classes related to the
+ * MapReduce framework upload tool.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+package org.apache.hadoop.mapred.uploader;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b78607a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/test/java/org/apache/hadoop/mapred/uploader/TestFrameworkUploader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/test/java/org/apache/hadoop/mapred/uploader/TestFrameworkUploader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/test/java/org/apache/hadoop/mapred/uploader/TestFrameworkUploader.java
new file mode 100644
index 0000000..9d03165
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-uploader/src/test/java/org/apache/hadoop/mapred/uploader/TestFrameworkUploader.java
@@ -0,0 +1,315 @@
+/**
+ * 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.mapred.uploader;
+
+import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
+import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
+import org.apache.commons.io.FileUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.zip.GZIPInputStream;
+
+/**
+ * Unit test class for FrameworkUploader.
+ */
+public class TestFrameworkUploader {
+  private static String testDir;
+
+  @Before
+  public void setUp() {
+    String testRootDir =
+        new File(System.getProperty("test.build.data", "/tmp"))
+            .getAbsolutePath()
+            .replace(' ', '+');
+    Random random = new Random(System.currentTimeMillis());
+    testDir = testRootDir + File.separatorChar +
+        Long.toString(random.nextLong());
+  }
+
+  /**
+   * Test requesting command line help.
+   * @throws IOException test failure
+   */
+  @Test
+  public void testHelp() throws IOException {
+    String[] args = new String[]{"-help"};
+    FrameworkUploader uploader = new FrameworkUploader();
+    boolean success = uploader.parseArguments(args);
+    Assert.assertFalse("Expected to print help", success);
+    Assert.assertEquals("Expected ignore run", null,
+        uploader.input);
+    Assert.assertEquals("Expected ignore run", null,
+        uploader.whitelist);
+    Assert.assertEquals("Expected ignore run", null,
+        uploader.target);
+  }
+
+  /**
+   * Test invalid argument parsing.
+   * @throws IOException test failure
+   */
+  @Test
+  public void testWrongArgument() throws IOException {
+    String[] args = new String[]{"-unexpected"};
+    FrameworkUploader uploader = new FrameworkUploader();
+    boolean success = uploader.parseArguments(args);
+    Assert.assertFalse("Expected to print help", success);
+  }
+
+  /**
+   * Test normal argument passing.
+   * @throws IOException test failure
+   */
+  @Test
+  public void testArguments() throws IOException {
+    String[] args =
+        new String[]{
+            "-input", "A",
+            "-whitelist", "B",
+            "-blacklist", "C",
+            "-fs", "hdfs://C:8020",
+            "-target", "D",
+            "-replication", "100"};
+    FrameworkUploader uploader = new FrameworkUploader();
+    boolean success = uploader.parseArguments(args);
+    Assert.assertTrue("Expected to print help", success);
+    Assert.assertEquals("Input mismatch", "A",
+        uploader.input);
+    Assert.assertEquals("Whitelist mismatch", "B",
+        uploader.whitelist);
+    Assert.assertEquals("Blacklist mismatch", "C",
+        uploader.blacklist);
+    Assert.assertEquals("Target mismatch", "hdfs://C:8020/D",
+        uploader.target);
+    Assert.assertEquals("Replication mismatch", 100,
+        uploader.replication);
+  }
+
+  /**
+   * Test whether we can filter a class path properly.
+   * @throws IOException test failure
+   */
+  @Test
+  public void testCollectPackages() throws IOException, UploaderException {
+    File parent = new File(testDir);
+    try {
+      parent.deleteOnExit();
+      Assert.assertTrue("Directory creation failed", parent.mkdirs());
+      File dirA = new File(parent, "A");
+      Assert.assertTrue(dirA.mkdirs());
+      File dirB = new File(parent, "B");
+      Assert.assertTrue(dirB.mkdirs());
+      File jarA = new File(dirA, "a.jar");
+      Assert.assertTrue(jarA.createNewFile());
+      File jarB = new File(dirA, "b.jar");
+      Assert.assertTrue(jarB.createNewFile());
+      File jarC = new File(dirA, "c.jar");
+      Assert.assertTrue(jarC.createNewFile());
+      File txtD = new File(dirA, "d.txt");
+      Assert.assertTrue(txtD.createNewFile());
+      File jarD = new File(dirB, "d.jar");
+      Assert.assertTrue(jarD.createNewFile());
+      File txtE = new File(dirB, "e.txt");
+      Assert.assertTrue(txtE.createNewFile());
+
+      FrameworkUploader uploader = new FrameworkUploader();
+      uploader.whitelist = ".*a\\.jar,.*b\\.jar,.*d\\.jar";
+      uploader.blacklist = ".*b\\.jar";
+      uploader.input = dirA.getAbsolutePath() + File.separatorChar + "*" +
+          File.pathSeparatorChar +
+          dirB.getAbsolutePath() + File.separatorChar + "*";
+      uploader.collectPackages();
+      Assert.assertEquals("Whitelist count error", 3,
+          uploader.whitelistedFiles.size());
+      Assert.assertEquals("Blacklist count error", 1,
+          uploader.blacklistedFiles.size());
+
+      Assert.assertTrue("File not collected",
+          uploader.filteredInputFiles.contains(jarA.getAbsolutePath()));
+      Assert.assertFalse("File collected",
+          uploader.filteredInputFiles.contains(jarB.getAbsolutePath()));
+      Assert.assertTrue("File not collected",
+          uploader.filteredInputFiles.contains(jarD.getAbsolutePath()));
+      Assert.assertEquals("Too many whitelists", 2,
+          uploader.filteredInputFiles.size());
+    } finally {
+      FileUtils.deleteDirectory(parent);
+    }
+  }
+
+  /**
+   * Test building a tarball from source jars.
+   */
+  @Test
+  public void testBuildTarBall() throws IOException, UploaderException {
+    File parent = new File(testDir);
+    try {
+      parent.deleteOnExit();
+      FrameworkUploader uploader = prepareTree(parent);
+
+      File gzipFile = new File("upload.tar.gz");
+      gzipFile.deleteOnExit();
+      Assert.assertTrue("Creating output", gzipFile.createNewFile());
+      uploader.targetStream = new FileOutputStream(gzipFile);
+
+      uploader.buildPackage();
+
+      TarArchiveInputStream result = null;
+      try {
+        result =
+            new TarArchiveInputStream(
+                new GZIPInputStream(new FileInputStream(gzipFile)));
+        Set<String> fileNames = new HashSet<>();
+        Set<Long> sizes = new HashSet<>();
+        TarArchiveEntry entry1 = result.getNextTarEntry();
+        fileNames.add(entry1.getName());
+        sizes.add(entry1.getSize());
+        TarArchiveEntry entry2 = result.getNextTarEntry();
+        fileNames.add(entry2.getName());
+        sizes.add(entry2.getSize());
+        Assert.assertTrue(
+            "File name error", fileNames.contains("a.jar"));
+        Assert.assertTrue(
+            "File size error", sizes.contains((long) 13));
+        Assert.assertTrue(
+            "File name error", fileNames.contains("b.jar"));
+        Assert.assertTrue(
+            "File size error", sizes.contains((long) 14));
+      } finally {
+        if (result != null) {
+          result.close();
+        }
+      }
+    } finally {
+      FileUtils.deleteDirectory(parent);
+    }
+  }
+
+  /**
+   * Test upload to HDFS.
+   */
+  @Test
+  public void testUpload() throws IOException, UploaderException {
+    final String fileName = "/upload.tar.gz";
+    File parent = new File(testDir);
+    try {
+      parent.deleteOnExit();
+
+      FrameworkUploader uploader = prepareTree(parent);
+
+      uploader.target = "file://" + parent.getAbsolutePath() + fileName;
+
+      uploader.buildPackage();
+      try (TarArchiveInputStream archiveInputStream = new TarArchiveInputStream(
+          new GZIPInputStream(
+              new FileInputStream(
+                  parent.getAbsolutePath() + fileName)))) {
+        Set<String> fileNames = new HashSet<>();
+        Set<Long> sizes = new HashSet<>();
+        TarArchiveEntry entry1 = archiveInputStream.getNextTarEntry();
+        fileNames.add(entry1.getName());
+        sizes.add(entry1.getSize());
+        TarArchiveEntry entry2 = archiveInputStream.getNextTarEntry();
+        fileNames.add(entry2.getName());
+        sizes.add(entry2.getSize());
+        Assert.assertTrue(
+            "File name error", fileNames.contains("a.jar"));
+        Assert.assertTrue(
+            "File size error", sizes.contains((long) 13));
+        Assert.assertTrue(
+            "File name error", fileNames.contains("b.jar"));
+        Assert.assertTrue(
+            "File size error", sizes.contains((long) 14));
+      }
+    } finally {
+      FileUtils.deleteDirectory(parent);
+    }
+  }
+
+  /**
+   * Prepare a mock directory tree to compress and upload.
+   */
+  private FrameworkUploader prepareTree(File parent)
+      throws FileNotFoundException {
+    Assert.assertTrue(parent.mkdirs());
+    File dirA = new File(parent, "A");
+    Assert.assertTrue(dirA.mkdirs());
+    File jarA = new File(parent, "a.jar");
+    PrintStream printStream = new PrintStream(new FileOutputStream(jarA));
+    printStream.println("Hello World!");
+    printStream.close();
+    File jarB = new File(dirA, "b.jar");
+    printStream = new PrintStream(new FileOutputStream(jarB));
+    printStream.println("Hello Galaxy!");
+    printStream.close();
+
+    FrameworkUploader uploader = new FrameworkUploader();
+    uploader.filteredInputFiles.add(jarA.getAbsolutePath());
+    uploader.filteredInputFiles.add(jarB.getAbsolutePath());
+
+    return uploader;
+  }
+
+  /**
+   * Test regex pattern matching and environment variable replacement.
+   */
+  @Test
+  public void testEnvironmentReplacement() throws UploaderException {
+    String input = "C/$A/B,$B,D";
+    Map<String, String> map = new HashMap<>();
+    map.put("A", "X");
+    map.put("B", "Y");
+    map.put("C", "Z");
+    FrameworkUploader uploader = new FrameworkUploader();
+    String output = uploader.expandEnvironmentVariables(input, map);
+    Assert.assertEquals("Environment not expanded", "C/X/B,Y,D", output);
+
+  }
+
+  /**
+   * Test regex pattern matching and environment variable replacement.
+   */
+  @Test
+  public void testRecursiveEnvironmentReplacement()
+      throws UploaderException {
+    String input = "C/$A/B,$B,D";
+    Map<String, String> map = new HashMap<>();
+    map.put("A", "X");
+    map.put("B", "$C");
+    map.put("C", "Y");
+    FrameworkUploader uploader = new FrameworkUploader();
+    String output = uploader.expandEnvironmentVariables(input, map);
+    Assert.assertEquals("Environment not expanded", "C/X/B,Y,D", output);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b78607a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
index 274a821..a8350cb 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
@@ -326,5 +326,6 @@
     <module>hadoop-mapreduce-client-hs</module>
     <module>hadoop-mapreduce-client-hs-plugins</module>
     <module>hadoop-mapreduce-client-nativetask</module>
+    <module>hadoop-mapreduce-client-uploader</module>
   </modules>
 </project>


---------------------------------------------------------------------
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-4813. TestRMWebServicesDelegationTokenAuthentication.testDoAs fails intermittently (grepas via rkanter)

Posted by ae...@apache.org.
YARN-4813. TestRMWebServicesDelegationTokenAuthentication.testDoAs fails intermittently (grepas via rkanter)


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

Branch: refs/heads/HDFS-7240
Commit: c83fe4491731c994a4867759d80db31d9c1cab60
Parents: 3b78607
Author: Robert Kanter <rk...@apache.org>
Authored: Fri Dec 1 12:18:13 2017 -0800
Committer: Robert Kanter <rk...@apache.org>
Committed: Fri Dec 1 12:18:13 2017 -0800

----------------------------------------------------------------------
 ...stRMWebServicesDelegationTokenAuthentication.java | 15 ++++++++++++++-
 1 file changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c83fe449/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesDelegationTokenAuthentication.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesDelegationTokenAuthentication.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesDelegationTokenAuthentication.java
index b406fdb..41e56ae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesDelegationTokenAuthentication.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesDelegationTokenAuthentication.java
@@ -76,6 +76,8 @@ public class TestRMWebServicesDelegationTokenAuthentication {
     TestRMWebServicesDelegationTokenAuthentication.class.getName() + "-root");
   private static File httpSpnegoKeytabFile = new File(
     KerberosTestUtils.getKeytabFile());
+  private static final String SUN_SECURITY_KRB5_RCACHE_KEY =
+      "sun.security.krb5.rcache";
 
   private static String httpSpnegoPrincipal = KerberosTestUtils
     .getServerPrincipal();
@@ -83,7 +85,7 @@ public class TestRMWebServicesDelegationTokenAuthentication {
   private static boolean miniKDCStarted = false;
   private static MiniKdc testMiniKDC;
   private static MockRM rm;
-
+  private static String sunSecurityKrb5RcacheValue;
 
   String delegationTokenHeader;
 
@@ -98,6 +100,11 @@ public class TestRMWebServicesDelegationTokenAuthentication {
   @BeforeClass
   public static void setUp() {
     try {
+      // Disabling kerberos replay cache to avoid "Request is a replay" errors
+      // caused by frequent webservice calls
+      sunSecurityKrb5RcacheValue =
+          System.getProperty(SUN_SECURITY_KRB5_RCACHE_KEY);
+      System.setProperty(SUN_SECURITY_KRB5_RCACHE_KEY, "none");
       testMiniKDC = new MiniKdc(MiniKdc.createConf(), testRootDir);
       setupKDC();
       setupAndStartRM();
@@ -114,6 +121,12 @@ public class TestRMWebServicesDelegationTokenAuthentication {
     if (rm != null) {
       rm.stop();
     }
+    if (sunSecurityKrb5RcacheValue == null) {
+      System.clearProperty(SUN_SECURITY_KRB5_RCACHE_KEY);
+    } else {
+      System.setProperty(SUN_SECURITY_KRB5_RCACHE_KEY,
+          sunSecurityKrb5RcacheValue);
+    }
   }
 
   @Parameterized.Parameters


---------------------------------------------------------------------
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-15039. Move SemaphoredDelegatingExecutor to hadoop-common. Contributed by Genmao Yu

Posted by ae...@apache.org.
HADOOP-15039. Move SemaphoredDelegatingExecutor to hadoop-common. 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/73b86979
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/73b86979
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/73b86979

Branch: refs/heads/HDFS-7240
Commit: 73b86979d661f4ad56fcfc3a05a403dfcb2a860e
Parents: 44b06d3
Author: Kai Zheng <zh...@alibaba-inc.com>
Authored: Wed Dec 6 12:01:36 2017 +0800
Committer: Kai Zheng <zh...@alibaba-inc.com>
Committed: Wed Dec 6 12:01:36 2017 +0800

----------------------------------------------------------------------
 .../fs/s3a/BlockingThreadPoolExecutorService.java    |  7 +++----
 .../java/org/apache/hadoop/fs/s3a/S3AFileSystem.java |  2 ++
 .../hadoop/fs/s3a/SemaphoredDelegatingExecutor.java  | 15 ++++++---------
 .../s3a/ITestBlockingThreadPoolExecutorService.java  |  2 ++
 4 files changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/73b86979/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 f13942d..404eea9 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
@@ -15,8 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-package org.apache.hadoop.fs.s3a;
+package org.apache.hadoop.util;
 
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -42,7 +41,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
  * this s4 threadpool</a>
  */
 @InterfaceAudience.Private
-final class BlockingThreadPoolExecutorService
+public final class BlockingThreadPoolExecutorService
     extends SemaphoredDelegatingExecutor {
 
   private static final Logger LOG = LoggerFactory
@@ -86,7 +85,7 @@ final class BlockingThreadPoolExecutorService
    * @return a thread factory that creates named, daemon threads with
    * the supplied exception handler and normal priority
    */
-  static ThreadFactory newDaemonThreadFactory(final String prefix) {
+  public 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/73b86979/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 63a4349..e927758 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
@@ -112,8 +112,10 @@ import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.Invoker.*;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73b86979/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SemaphoredDelegatingExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SemaphoredDelegatingExecutor.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SemaphoredDelegatingExecutor.java
index 6b21912..bcc19e3 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SemaphoredDelegatingExecutor.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SemaphoredDelegatingExecutor.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.s3a;
+package org.apache.hadoop.util;
 
 import com.google.common.util.concurrent.ForwardingListeningExecutorService;
 import com.google.common.util.concurrent.Futures;
@@ -42,17 +42,13 @@ import java.util.concurrent.TimeoutException;
  * This is a refactoring of {@link BlockingThreadPoolExecutorService}; that code
  * contains the thread pool logic, whereas this isolates the semaphore
  * and submit logic for use with other thread pools and delegation models.
- * In particular, it <i>permits multiple per stream executors to share a
- * single per-FS-instance executor; the latter to throttle overall
- * load from the the FS, the others to limit the amount of load which
- * a single output stream can generate.</i>
  * <p>
  * This is inspired by <a href="https://github.com/apache/incubator-s4/blob/master/subprojects/s4-comm/src/main/java/org/apache/s4/comm/staging/BlockingThreadPoolExecutorService.java">
  * this s4 threadpool</a>
  */
 @SuppressWarnings("NullableProblems")
 @InterfaceAudience.Private
-class SemaphoredDelegatingExecutor extends
+public class SemaphoredDelegatingExecutor extends
     ForwardingListeningExecutorService {
 
   private final Semaphore queueingPermits;
@@ -65,7 +61,8 @@ class SemaphoredDelegatingExecutor extends
    * @param permitCount number of permits into the queue permitted
    * @param fair should the semaphore be "fair"
    */
-  SemaphoredDelegatingExecutor(ListeningExecutorService executorDelegatee,
+  public SemaphoredDelegatingExecutor(
+      ListeningExecutorService executorDelegatee,
       int permitCount,
       boolean fair) {
     this.permitCount = permitCount;
@@ -190,7 +187,7 @@ class SemaphoredDelegatingExecutor extends
 
     private Runnable delegatee;
 
-    public RunnableWithPermitRelease(Runnable delegatee) {
+    RunnableWithPermitRelease(Runnable delegatee) {
       this.delegatee = delegatee;
     }
 
@@ -212,7 +209,7 @@ class SemaphoredDelegatingExecutor extends
 
     private Callable<T> delegatee;
 
-    public CallableWithPermitRelease(Callable<T> delegatee) {
+    CallableWithPermitRelease(Callable<T> delegatee) {
       this.delegatee = delegatee;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73b86979/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java
index b1b8240..3dfe286 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestBlockingThreadPoolExecutorService.java
@@ -19,6 +19,8 @@
 package org.apache.hadoop.fs.s3a;
 
 import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
+import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
 import org.apache.hadoop.util.StopWatch;
 
 import org.junit.AfterClass;


---------------------------------------------------------------------
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: HDFS-11754. Make FsServerDefaults cache configurable. Contributed by Mikhail Erofeev.

Posted by ae...@apache.org.
HDFS-11754. Make FsServerDefaults cache configurable. Contributed by Mikhail Erofeev.


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

Branch: refs/heads/HDFS-7240
Commit: 53509f295b5274059541565d7216bf98aa35347d
Parents: 8498d28
Author: Kihwal Lee <ki...@apache.org>
Authored: Wed Nov 29 15:11:14 2017 -0600
Committer: Kihwal Lee <ki...@apache.org>
Committed: Wed Nov 29 15:11:14 2017 -0600

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  10 +-
 .../hdfs/client/HdfsClientConfigKeys.java       |   4 +
 .../src/main/resources/hdfs-default.xml         |  10 ++
 .../apache/hadoop/hdfs/TestFileCreation.java    | 103 +++++++++++++++++++
 4 files changed, 124 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/53509f29/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 5652321..25e0f6c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -24,6 +24,8 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACH
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT_DEFAULT;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_LOCAL_INTERFACES;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_DEFAULT;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY;
 
@@ -205,8 +207,6 @@ import com.google.common.net.InetAddresses;
 public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     DataEncryptionKeyFactory {
   public static final Logger LOG = LoggerFactory.getLogger(DFSClient.class);
-  // 1 hour
-  public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L;
   private static final String DFS_KMS_PREFIX = "dfs-kms-";
 
   private final Configuration conf;
@@ -240,6 +240,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
   private static volatile ThreadPoolExecutor STRIPED_READ_THREAD_POOL;
   private final int smallBufferSize;
+  private final long serverDefaultsValidityPeriod;
 
   public DfsClientConf getConf() {
     return dfsClientConf;
@@ -371,6 +372,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
             null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_READS, false);
     Long readahead = (conf.get(DFS_CLIENT_CACHE_READAHEAD) == null) ?
         null : conf.getLong(DFS_CLIENT_CACHE_READAHEAD, 0);
+    this.serverDefaultsValidityPeriod =
+            conf.getLong(DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_KEY,
+      DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_DEFAULT);
     Boolean writeDropBehind =
         (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_WRITES) == null) ?
             null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_WRITES, false);
@@ -663,7 +667,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     checkOpen();
     long now = Time.monotonicNow();
     if ((serverDefaults == null) ||
-        (now - serverDefaultsLastUpdate > SERVER_DEFAULTS_VALIDITY_PERIOD)) {
+        (now - serverDefaultsLastUpdate > serverDefaultsValidityPeriod)) {
       serverDefaults = namenode.getServerDefaults();
       serverDefaultsLastUpdate = now;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53509f29/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index 97cb68b..d6c47b5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -112,6 +112,10 @@ public interface HdfsClientConfigKeys {
   String  DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY =
       "dfs.client.max.block.acquire.failures";
   int     DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT = 3;
+  String  DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_KEY =
+      "dfs.client.server-defaults.validity.period.ms";
+  long    DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_DEFAULT =
+      TimeUnit.HOURS.toMillis(1);
   String  DFS_CHECKSUM_TYPE_KEY = "dfs.checksum.type";
   String  DFS_CHECKSUM_TYPE_DEFAULT = "CRC32C";
   String  DFS_BYTES_PER_CHECKSUM_KEY = "dfs.bytes-per-checksum";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53509f29/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 7ff91f2..79c2d8e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -2283,6 +2283,16 @@
 </property>
 
 <property>
+  <name>dfs.client.server-defaults.validity.period.ms</name>
+  <value>3600000</value>
+  <description>
+    The amount of milliseconds after which cached server defaults are updated.
+
+    By default this parameter is set to 1 hour.
+  </description>
+</property>
+
+<property>
   <name>dfs.namenode.enable.retrycache</name>
   <value>true</value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53509f29/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
index a738343..89aa9ba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
@@ -23,6 +23,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_KEY;
@@ -38,6 +39,7 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.junit.Assume.assumeTrue;
+import static org.mockito.Mockito.doReturn;
 
 import java.io.BufferedReader;
 import java.io.ByteArrayOutputStream;
@@ -51,6 +53,7 @@ import java.net.URI;
 import java.net.UnknownHostException;
 import java.security.PrivilegedExceptionAction;
 import java.util.EnumSet;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -179,6 +182,106 @@ public class TestFileCreation {
     }
   }
 
+  /**
+   * Test that server default values are cached on the client size
+   * and are stale after namenode update.
+   */
+  @Test
+  public void testServerDefaultsWithCaching()
+      throws IOException, InterruptedException {
+    // Create cluster with an explicit block size param
+    Configuration clusterConf = new HdfsConfiguration();
+    long originalBlockSize = DFS_BLOCK_SIZE_DEFAULT * 2;
+    clusterConf.setLong(DFS_BLOCK_SIZE_KEY, originalBlockSize);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(clusterConf)
+        .numDataNodes(0)
+        .build();
+    cluster.waitActive();
+    // Set a spy namesystem inside the namenode and return it
+    FSNamesystem spyNamesystem =
+        NameNodeAdapter.spyOnNamesystem(cluster.getNameNode());
+    InetSocketAddress nameNodeAddr = cluster.getNameNode().getNameNodeAddress();
+    try {
+      // Create a dfs client and set a long enough validity interval
+      Configuration clientConf = new HdfsConfiguration();
+      clientConf.setLong(DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_KEY,
+          TimeUnit.MINUTES.toMillis(1));
+      DFSClient dfsClient = new DFSClient(nameNodeAddr, clientConf);
+      FsServerDefaults defaults = dfsClient.getServerDefaults();
+      assertEquals(originalBlockSize, defaults.getBlockSize());
+
+      // Update the namenode with a new parameter
+      long updatedDefaultBlockSize = DFS_BLOCK_SIZE_DEFAULT * 3;
+      FsServerDefaults newDefaults =
+          new FsServerDefaults(updatedDefaultBlockSize,
+              defaults.getBytesPerChecksum(), defaults.getWritePacketSize(),
+              defaults.getReplication(), defaults.getFileBufferSize(),
+              defaults.getEncryptDataTransfer(), defaults.getTrashInterval(),
+              defaults.getChecksumType(), defaults.getKeyProviderUri(),
+              defaults.getDefaultStoragePolicyId());
+      doReturn(newDefaults).when(spyNamesystem).getServerDefaults();
+
+      // The value is stale
+      Thread.sleep(1);
+      defaults = dfsClient.getServerDefaults();
+      assertEquals(originalBlockSize, defaults.getBlockSize());
+
+      // Another client reads the updated value correctly
+      DFSClient newDfsClient = new DFSClient(nameNodeAddr, clientConf);
+      defaults = newDfsClient.getServerDefaults();
+      assertEquals(updatedDefaultBlockSize, defaults.getBlockSize());
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Test that server defaults are updated on the client after cache expiration.
+   */
+  @Test
+  public void testServerDefaultsWithMinimalCaching()
+      throws IOException, InterruptedException {
+    // Create cluster with an explicit block size param
+    Configuration clusterConf = new HdfsConfiguration();
+    long originalBlockSize = DFS_BLOCK_SIZE_DEFAULT * 2;
+    clusterConf.setLong(DFS_BLOCK_SIZE_KEY, originalBlockSize);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(clusterConf)
+        .numDataNodes(0)
+        .build();
+    cluster.waitActive();
+    // Set a spy namesystem inside the namenode and return it
+    FSNamesystem spyNamesystem =
+        NameNodeAdapter.spyOnNamesystem(cluster.getNameNode());
+    InetSocketAddress nameNodeAddr = cluster.getNameNode().getNameNodeAddress();
+    try {
+      // Create a dfs client and set a minimal validity interval
+      Configuration clientConf = new HdfsConfiguration();
+      // Invalidate cache in at most 1 ms, see DfsClient#getServerDefaults
+      clientConf.setLong(DFS_CLIENT_SERVER_DEFAULTS_VALIDITY_PERIOD_MS_KEY, 0L);
+      DFSClient dfsClient = new DFSClient(nameNodeAddr, clientConf);
+      FsServerDefaults defaults = dfsClient.getServerDefaults();
+      assertEquals(originalBlockSize, defaults.getBlockSize());
+
+      // Update the namenode with a new FsServerDefaults
+      long updatedDefaultBlockSize = DFS_BLOCK_SIZE_DEFAULT * 3;
+      FsServerDefaults newDefaults =
+          new FsServerDefaults(updatedDefaultBlockSize,
+              defaults.getBytesPerChecksum(), defaults.getWritePacketSize(),
+              defaults.getReplication(), defaults.getFileBufferSize(),
+              defaults.getEncryptDataTransfer(), defaults.getTrashInterval(),
+              defaults.getChecksumType(), defaults.getKeyProviderUri(),
+              defaults.getDefaultStoragePolicyId());
+      doReturn(newDefaults).when(spyNamesystem).getServerDefaults();
+
+      Thread.sleep(1);
+      defaults = dfsClient.getServerDefaults();
+      // Value is updated correctly
+      assertEquals(updatedDefaultBlockSize, defaults.getBlockSize());
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
   @Test
   public void testFileCreation() throws IOException {
     checkFileCreation(null, false);


---------------------------------------------------------------------
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: YARN-6507. Add support in NodeManager to isolate FPGA devices with CGroups. (Zhankun Tang via wangda)

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/7225ec0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/fpga/TestFpgaResourceHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/fpga/TestFpgaResourceHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/fpga/TestFpgaResourceHandler.java
new file mode 100644
index 0000000..d3d55fa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/fpga/TestFpgaResourceHandler.java
@@ -0,0 +1,458 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.fpga;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.*;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ResourceMappings;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationException;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationExecutor;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.CGroupsHandler;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandlerException;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceSet;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga.FpgaDiscoverer;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga.IntelFpgaOpenclPlugin;
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
+import org.apache.hadoop.yarn.util.resource.TestResourceUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.mockito.Mockito.*;
+
+
+public class TestFpgaResourceHandler {
+  private Context mockContext;
+  private FpgaResourceHandlerImpl fpgaResourceHandler;
+  private Configuration configuration;
+  private CGroupsHandler mockCGroupsHandler;
+  private PrivilegedOperationExecutor mockPrivilegedExecutor;
+  private NMStateStoreService mockNMStateStore;
+  private ConcurrentHashMap<ContainerId, Container> runningContainersMap;
+  private IntelFpgaOpenclPlugin mockVendorPlugin;
+  private static final String vendorType = "IntelOpenCL";
+
+  @Before
+  public void setup() {
+    TestResourceUtils.addNewTypesToResources(ResourceInformation.FPGA_URI);
+    configuration = new YarnConfiguration();
+
+    mockCGroupsHandler = mock(CGroupsHandler.class);
+    mockPrivilegedExecutor = mock(PrivilegedOperationExecutor.class);
+    mockNMStateStore = mock(NMStateStoreService.class);
+    mockContext = mock(Context.class);
+    // Assumed devices parsed from output
+    List<FpgaResourceAllocator.FpgaDevice> list = new ArrayList<>();
+    list.add(new FpgaResourceAllocator.FpgaDevice(vendorType, 247, 0, null));
+    list.add(new FpgaResourceAllocator.FpgaDevice(vendorType, 247, 1, null));
+    list.add(new FpgaResourceAllocator.FpgaDevice(vendorType, 247, 2, null));
+    list.add(new FpgaResourceAllocator.FpgaDevice(vendorType, 247, 3, null));
+    list.add(new FpgaResourceAllocator.FpgaDevice(vendorType, 247, 4, null));
+    mockVendorPlugin = mockPlugin(vendorType, list);
+    FpgaDiscoverer.getInstance().setConf(configuration);
+    when(mockContext.getNMStateStore()).thenReturn(mockNMStateStore);
+    runningContainersMap = new ConcurrentHashMap<>();
+    when(mockContext.getContainers()).thenReturn(runningContainersMap);
+
+    fpgaResourceHandler = new FpgaResourceHandlerImpl(mockContext,
+        mockCGroupsHandler, mockPrivilegedExecutor, mockVendorPlugin);
+  }
+
+  @Test
+  public void testBootstrap() throws ResourceHandlerException {
+    // Case 1. auto
+    String allowed = "auto";
+    configuration.set(YarnConfiguration.NM_FPGA_ALLOWED_DEVICES, allowed);
+    fpgaResourceHandler.bootstrap(configuration);
+    verify(mockVendorPlugin, times(1)).initPlugin(configuration);
+    verify(mockCGroupsHandler, times(1)).initializeCGroupController(
+        CGroupsHandler.CGroupController.DEVICES);
+    Assert.assertEquals(5, fpgaResourceHandler.getFpgaAllocator().getAvailableFpgaCount());
+    Assert.assertEquals(5, fpgaResourceHandler.getFpgaAllocator().getAllowedFpga().size());
+    // Case 2. subset of devices
+    fpgaResourceHandler = new FpgaResourceHandlerImpl(mockContext,
+        mockCGroupsHandler, mockPrivilegedExecutor, mockVendorPlugin);
+    allowed = "0,1,2";
+    configuration.set(YarnConfiguration.NM_FPGA_ALLOWED_DEVICES, allowed);
+    fpgaResourceHandler.bootstrap(configuration);
+    Assert.assertEquals(3, fpgaResourceHandler.getFpgaAllocator().getAllowedFpga().size());
+    List<FpgaResourceAllocator.FpgaDevice> allowedDevices = fpgaResourceHandler.getFpgaAllocator().getAllowedFpga();
+    for (String s : allowed.split(",")) {
+      boolean check = false;
+      for (FpgaResourceAllocator.FpgaDevice device : allowedDevices) {
+        if (device.getMinor().toString().equals(s)) {
+          check = true;
+        }
+      }
+      Assert.assertTrue("Minor:" + s +"found", check);
+    }
+    Assert.assertEquals(3, fpgaResourceHandler.getFpgaAllocator().getAvailableFpgaCount());
+
+    // Case 3. User configuration contains invalid minor device number
+    fpgaResourceHandler = new FpgaResourceHandlerImpl(mockContext,
+        mockCGroupsHandler, mockPrivilegedExecutor, mockVendorPlugin);
+    allowed = "0,1,7";
+    configuration.set(YarnConfiguration.NM_FPGA_ALLOWED_DEVICES, allowed);
+    fpgaResourceHandler.bootstrap(configuration);
+    Assert.assertEquals(2, fpgaResourceHandler.getFpgaAllocator().getAvailableFpgaCount());
+    Assert.assertEquals(2, fpgaResourceHandler.getFpgaAllocator().getAllowedFpga().size());
+  }
+
+  @Test
+  public void testBootstrapWithInvalidUserConfiguration() throws ResourceHandlerException {
+    // User configuration contains invalid minor device number
+    String allowed = "0,1,7";
+    configuration.set(YarnConfiguration.NM_FPGA_ALLOWED_DEVICES, allowed);
+    fpgaResourceHandler.bootstrap(configuration);
+    Assert.assertEquals(2, fpgaResourceHandler.getFpgaAllocator().getAllowedFpga().size());
+    Assert.assertEquals(2, fpgaResourceHandler.getFpgaAllocator().getAvailableFpgaCount());
+
+    String[] invalidAllowedStrings = {"a,1,2,", "a,1,2", "0,1,2,#", "a", "1,"};
+    for (String s : invalidAllowedStrings) {
+      boolean invalidConfiguration = false;
+      configuration.set(YarnConfiguration.NM_FPGA_ALLOWED_DEVICES, s);
+      try {
+        fpgaResourceHandler.bootstrap(configuration);
+      } catch (ResourceHandlerException e) {
+        invalidConfiguration = true;
+      }
+      Assert.assertTrue(invalidConfiguration);
+    }
+
+    String[] allowedStrings = {"1,2", "1"};
+    for (String s : allowedStrings) {
+      boolean invalidConfiguration = false;
+      configuration.set(YarnConfiguration.NM_FPGA_ALLOWED_DEVICES, s);
+      try {
+        fpgaResourceHandler.bootstrap(configuration);
+      } catch (ResourceHandlerException e) {
+        invalidConfiguration = true;
+      }
+      Assert.assertFalse(invalidConfiguration);
+    }
+  }
+
+  @Test
+  public void testBootStrapWithEmptyUserConfiguration() throws ResourceHandlerException {
+    // User configuration contains invalid minor device number
+    String allowed = "";
+    boolean invalidConfiguration = false;
+    configuration.set(YarnConfiguration.NM_FPGA_ALLOWED_DEVICES, allowed);
+    try {
+      fpgaResourceHandler.bootstrap(configuration);
+    } catch (ResourceHandlerException e) {
+      invalidConfiguration = true;
+    }
+    Assert.assertTrue(invalidConfiguration);
+  }
+
+  @Test
+  public void testAllocationWithPreference() throws ResourceHandlerException, PrivilegedOperationException {
+    configuration.set(YarnConfiguration.NM_FPGA_ALLOWED_DEVICES, "0,1,2");
+    fpgaResourceHandler.bootstrap(configuration);
+    // Case 1. The id-0 container request 1 FPGA of IntelOpenCL type and GEMM IP
+    fpgaResourceHandler.preStart(mockContainer(0, 1, "GEMM"));
+    Assert.assertEquals(1, fpgaResourceHandler.getFpgaAllocator().getUsedFpgaCount());
+    verifyDeniedDevices(getContainerId(0), Arrays.asList(1, 2));
+    List<FpgaResourceAllocator.FpgaDevice> list = fpgaResourceHandler.getFpgaAllocator()
+        .getUsedFpga().get(getContainerId(0).toString());
+    for (FpgaResourceAllocator.FpgaDevice device : list) {
+      Assert.assertEquals("IP should be updated to GEMM", "GEMM", device.getIPID());
+    }
+    // Case 2. The id-1 container request 3 FPGA of IntelOpenCL and GEMM IP. this should fail
+    boolean flag = false;
+    try {
+      fpgaResourceHandler.preStart(mockContainer(1, 3, "GZIP"));
+    } catch (ResourceHandlerException e) {
+      flag = true;
+    }
+    Assert.assertTrue(flag);
+    // Case 3. Release the id-0 container
+    fpgaResourceHandler.postComplete(getContainerId(0));
+    Assert.assertEquals(0, fpgaResourceHandler.getFpgaAllocator().getUsedFpgaCount());
+    Assert.assertEquals(3, fpgaResourceHandler.getFpgaAllocator().getAvailableFpgaCount());
+    // Now we have enough devices, re-allocate for the id-1 container
+    fpgaResourceHandler.preStart(mockContainer(1, 3, "GEMM"));
+    // Id-1 container should have 0 denied devices
+    verifyDeniedDevices(getContainerId(1), new ArrayList<>());
+    Assert.assertEquals(3, fpgaResourceHandler.getFpgaAllocator().getUsedFpgaCount());
+    Assert.assertEquals(0, fpgaResourceHandler.getFpgaAllocator().getAvailableFpgaCount());
+    // Release container id-1
+    fpgaResourceHandler.postComplete(getContainerId(1));
+    Assert.assertEquals(0, fpgaResourceHandler.getFpgaAllocator().getUsedFpgaCount());
+    Assert.assertEquals(3, fpgaResourceHandler.getFpgaAllocator().getAvailableFpgaCount());
+    // Case 4. Now all 3 devices should have IPID GEMM
+    // Try container id-2 and id-3
+    fpgaResourceHandler.preStart(mockContainer(2, 1, "GZIP"));
+    fpgaResourceHandler.postComplete(getContainerId(2));
+    fpgaResourceHandler.preStart(mockContainer(3, 2, "GEMM"));
+
+    // IPID should be GEMM for id-3 container
+    list = fpgaResourceHandler.getFpgaAllocator()
+        .getUsedFpga().get(getContainerId(3).toString());
+    for (FpgaResourceAllocator.FpgaDevice device : list) {
+      Assert.assertEquals("IPID should be GEMM", "GEMM", device.getIPID());
+    }
+    Assert.assertEquals(2, fpgaResourceHandler.getFpgaAllocator().getUsedFpgaCount());
+    Assert.assertEquals(1, fpgaResourceHandler.getFpgaAllocator().getAvailableFpgaCount());
+    fpgaResourceHandler.postComplete(getContainerId(3));
+    Assert.assertEquals(0, fpgaResourceHandler.getFpgaAllocator().getUsedFpgaCount());
+    Assert.assertEquals(3, fpgaResourceHandler.getFpgaAllocator().getAvailableFpgaCount());
+
+    // Case 5. id-4 request 0 FPGA device
+    fpgaResourceHandler.preStart(mockContainer(4, 0, ""));
+    // Deny all devices for id-4
+    verifyDeniedDevices(getContainerId(4), Arrays.asList(0, 1, 2));
+    Assert.assertEquals(0, fpgaResourceHandler.getFpgaAllocator().getUsedFpgaCount());
+    Assert.assertEquals(3, fpgaResourceHandler.getFpgaAllocator().getAvailableFpgaCount());
+
+    // Case 6. id-5 with invalid FPGA device
+    try {
+      fpgaResourceHandler.preStart(mockContainer(5, -2, ""));
+    } catch (ResourceHandlerException e) {
+      Assert.assertTrue(true);
+    }
+  }
+
+  @Test
+  public void testsAllocationWithExistingIPIDDevices() throws ResourceHandlerException, PrivilegedOperationException {
+    configuration.set(YarnConfiguration.NM_FPGA_ALLOWED_DEVICES, "0,1,2");
+    fpgaResourceHandler.bootstrap(configuration);
+    // The id-0 container request 3 FPGA of IntelOpenCL type and GEMM IP
+    fpgaResourceHandler.preStart(mockContainer(0, 3, "GEMM"));
+    Assert.assertEquals(3, fpgaResourceHandler.getFpgaAllocator().getUsedFpgaCount());
+    List<FpgaResourceAllocator.FpgaDevice> list = fpgaResourceHandler.getFpgaAllocator()
+        .getUsedFpga().get(getContainerId(0).toString());
+    fpgaResourceHandler.postComplete(getContainerId(0));
+    for (FpgaResourceAllocator.FpgaDevice device : list) {
+      Assert.assertEquals("IP should be updated to GEMM", "GEMM", device.getIPID());
+    }
+
+    // Case 1. id-1 container request preStart, with no plugin.configureIP called
+    fpgaResourceHandler.preStart(mockContainer(1, 1, "GEMM"));
+    fpgaResourceHandler.preStart(mockContainer(2, 1, "GEMM"));
+    // we should have 3 times due to id-1 skip 1 invocation
+    verify(mockVendorPlugin, times(3)).configureIP(anyString(),anyString());
+    fpgaResourceHandler.postComplete(getContainerId(1));
+    fpgaResourceHandler.postComplete(getContainerId(2));
+
+    // Case 2. id-2 container request preStart, with 1 plugin.configureIP called
+    fpgaResourceHandler.preStart(mockContainer(1, 1, "GZIP"));
+    // we should have 4 times invocation
+    verify(mockVendorPlugin, times(4)).configureIP(anyString(),anyString());
+  }
+
+  @Test
+  public void testAllocationWithZeroDevices() throws ResourceHandlerException, PrivilegedOperationException {
+    configuration.set(YarnConfiguration.NM_FPGA_ALLOWED_DEVICES, "0,1,2");
+    fpgaResourceHandler.bootstrap(configuration);
+    // The id-0 container request 0 FPGA
+    fpgaResourceHandler.preStart(mockContainer(0, 0, null));
+    verifyDeniedDevices(getContainerId(0), Arrays.asList(0, 1, 2));
+    verify(mockVendorPlugin, times(0)).downloadIP(anyString(), anyString(), anyMap());
+    verify(mockVendorPlugin, times(0)).configureIP(anyString(), anyString());
+  }
+
+  @Test
+  public void testStateStore() throws ResourceHandlerException, IOException {
+    // Case 1. store 3 devices
+    configuration.set(YarnConfiguration.NM_FPGA_ALLOWED_DEVICES, "0,1,2");
+    fpgaResourceHandler.bootstrap(configuration);
+    Container container0 = mockContainer(0, 3, "GEMM");
+    fpgaResourceHandler.preStart(container0);
+    List<FpgaResourceAllocator.FpgaDevice> assigned =
+        fpgaResourceHandler.getFpgaAllocator().getUsedFpga().get(getContainerId(0).toString());
+    verify(mockNMStateStore).storeAssignedResources(container0,
+        ResourceInformation.FPGA_URI,
+        new ArrayList<>(assigned));
+    fpgaResourceHandler.postComplete(getContainerId(0));
+    // Case 2. ask 0, no store api called
+    Container container1 = mockContainer(1, 0, "");
+    fpgaResourceHandler.preStart(container1);
+    verify(mockNMStateStore, never()).storeAssignedResources(
+        eq(container1), eq(ResourceInformation.FPGA_URI), anyList());
+  }
+
+  @Test
+  public void testReacquireContainer() throws ResourceHandlerException {
+
+    Container c0 = mockContainer(0, 2, "GEMM");
+    List<FpgaResourceAllocator.FpgaDevice> assigned = new ArrayList<>();
+    assigned.add(new FpgaResourceAllocator.FpgaDevice(vendorType, 247, 0, null));
+    assigned.add(new FpgaResourceAllocator.FpgaDevice(vendorType, 247, 1, null));
+    // Mock we've stored the c0 states
+    mockStateStoreForContainer(c0, assigned);
+    // NM start
+    configuration.set(YarnConfiguration.NM_FPGA_ALLOWED_DEVICES, "0,1,2");
+    fpgaResourceHandler.bootstrap(configuration);
+    Assert.assertEquals(0, fpgaResourceHandler.getFpgaAllocator().getUsedFpgaCount());
+    Assert.assertEquals(3, fpgaResourceHandler.getFpgaAllocator().getAvailableFpgaCount());
+    // Case 1. try recover state for id-0 container
+    fpgaResourceHandler.reacquireContainer(getContainerId(0));
+    // minor number matches
+    List<FpgaResourceAllocator.FpgaDevice> used = fpgaResourceHandler.getFpgaAllocator().
+        getUsedFpga().get(getContainerId(0).toString());
+    int count = 0;
+    for (FpgaResourceAllocator.FpgaDevice device : used) {
+      if (device.getMinor().equals(0)){
+        count++;
+      }
+      if (device.getMinor().equals(1)) {
+        count++;
+      }
+    }
+    Assert.assertEquals("Unexpected used minor number in allocator",2, count);
+    List<FpgaResourceAllocator.FpgaDevice> available = fpgaResourceHandler.getFpgaAllocator().
+        getAvailableFpga().get(vendorType);
+    count = 0;
+    for (FpgaResourceAllocator.FpgaDevice device : available) {
+      if (device.getMinor().equals(2)) {
+        count++;
+      }
+    }
+    Assert.assertEquals("Unexpected available minor number in allocator", 1, count);
+
+
+    // Case 2. Recover a not allowed device with minor number 5
+    Container c1 = mockContainer(1, 1, "GEMM");
+    assigned = new ArrayList<>();
+    assigned.add(new FpgaResourceAllocator.FpgaDevice(vendorType, 247, 5, null));
+    // Mock we've stored the c1 states
+    mockStateStoreForContainer(c1, assigned);
+    boolean flag = false;
+    try {
+      fpgaResourceHandler.reacquireContainer(getContainerId(1));
+    } catch (ResourceHandlerException e) {
+      flag = true;
+    }
+    Assert.assertTrue(flag);
+    Assert.assertEquals(2, fpgaResourceHandler.getFpgaAllocator().getUsedFpgaCount());
+    Assert.assertEquals(1, fpgaResourceHandler.getFpgaAllocator().getAvailableFpgaCount());
+
+    // Case 3. recover a already used device by other container
+    Container c2 = mockContainer(2, 1, "GEMM");
+    assigned = new ArrayList<>();
+    assigned.add(new FpgaResourceAllocator.FpgaDevice(vendorType, 247, 1, null));
+    // Mock we've stored the c2 states
+    mockStateStoreForContainer(c2, assigned);
+    flag = false;
+    try {
+      fpgaResourceHandler.reacquireContainer(getContainerId(2));
+    } catch (ResourceHandlerException e) {
+      flag = true;
+    }
+    Assert.assertTrue(flag);
+    Assert.assertEquals(2, fpgaResourceHandler.getFpgaAllocator().getUsedFpgaCount());
+    Assert.assertEquals(1, fpgaResourceHandler.getFpgaAllocator().getAvailableFpgaCount());
+
+    // Case 4. recover a normal container c3 with remaining minor device number 2
+    Container c3 = mockContainer(3, 1, "GEMM");
+    assigned = new ArrayList<>();
+    assigned.add(new FpgaResourceAllocator.FpgaDevice(vendorType, 247, 2, null));
+    // Mock we've stored the c2 states
+    mockStateStoreForContainer(c3, assigned);
+    fpgaResourceHandler.reacquireContainer(getContainerId(3));
+    Assert.assertEquals(3, fpgaResourceHandler.getFpgaAllocator().getUsedFpgaCount());
+    Assert.assertEquals(0, fpgaResourceHandler.getFpgaAllocator().getAvailableFpgaCount());
+  }
+
+  private void verifyDeniedDevices(ContainerId containerId,
+      List<Integer> deniedDevices)
+      throws ResourceHandlerException, PrivilegedOperationException {
+    verify(mockCGroupsHandler, atLeastOnce()).createCGroup(
+        CGroupsHandler.CGroupController.DEVICES, containerId.toString());
+
+    if (null != deniedDevices && !deniedDevices.isEmpty()) {
+      verify(mockPrivilegedExecutor, times(1)).executePrivilegedOperation(
+          new PrivilegedOperation(PrivilegedOperation.OperationType.FPGA, Arrays
+              .asList(FpgaResourceHandlerImpl.CONTAINER_ID_CLI_OPTION,
+                  containerId.toString(),
+                  FpgaResourceHandlerImpl.EXCLUDED_FPGAS_CLI_OPTION,
+                  StringUtils.join(",", deniedDevices))), true);
+    } else if (deniedDevices.isEmpty()) {
+      verify(mockPrivilegedExecutor, times(1)).executePrivilegedOperation(
+          new PrivilegedOperation(PrivilegedOperation.OperationType.FPGA, Arrays
+              .asList(FpgaResourceHandlerImpl.CONTAINER_ID_CLI_OPTION,
+                  containerId.toString())), true);
+    }
+  }
+
+  private static IntelFpgaOpenclPlugin mockPlugin(String type, List<FpgaResourceAllocator.FpgaDevice> list) {
+    IntelFpgaOpenclPlugin plugin = mock(IntelFpgaOpenclPlugin.class);
+    when(plugin.initPlugin(Mockito.anyObject())).thenReturn(true);
+    when(plugin.getFpgaType()).thenReturn(type);
+    when(plugin.downloadIP(Mockito.anyString(), Mockito.anyString(), Mockito.anyMap())).thenReturn("/tmp");
+    when(plugin.configureIP(Mockito.anyString(), Mockito.anyObject())).thenReturn(true);
+    when(plugin.discover(Mockito.anyInt())).thenReturn(list);
+    return plugin;
+  }
+
+
+  private static Container mockContainer(int id, int numFpga, String IPID) {
+    Container c = mock(Container.class);
+
+    Resource res = Resource.newInstance(1024, 1);
+    ResourceMappings resMapping = new ResourceMappings();
+    res.setResourceValue(ResourceInformation.FPGA_URI, numFpga);
+    when(c.getResource()).thenReturn(res);
+    when(c.getResourceMappings()).thenReturn(resMapping);
+
+    when(c.getContainerId()).thenReturn(getContainerId(id));
+
+    ContainerLaunchContext clc = mock(ContainerLaunchContext.class);
+    Map<String, String> envs = new HashMap<>();
+    if (numFpga > 0) {
+      envs.put("REQUESTED_FPGA_IP_ID", IPID);
+    }
+    when(c.getLaunchContext()).thenReturn(clc);
+    when(clc.getEnvironment()).thenReturn(envs);
+    when(c.getWorkDir()).thenReturn("/tmp");
+    ResourceSet resourceSet = new ResourceSet();
+    when(c.getResourceSet()).thenReturn(resourceSet);
+
+    return c;
+  }
+
+  private void mockStateStoreForContainer(Container container,
+      List<FpgaResourceAllocator.FpgaDevice> assigned) {
+    ResourceMappings rmap = new ResourceMappings();
+    ResourceMappings.AssignedResources ar =
+        new ResourceMappings.AssignedResources();
+    ar.updateAssignedResources(new ArrayList<>(assigned));
+    rmap.addAssignedResources(ResourceInformation.FPGA_URI, ar);
+    when(container.getResourceMappings()).thenReturn(rmap);
+    runningContainersMap.put(container.getContainerId(), container);
+  }
+
+  private static ContainerId getContainerId(int id) {
+    return ContainerId.newContainerId(ApplicationAttemptId
+        .newInstance(ApplicationId.newInstance(1234L, 1), 1), id);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7225ec0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/TestFpgaDiscoverer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/TestFpgaDiscoverer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/TestFpgaDiscoverer.java
new file mode 100644
index 0000000..87fb4e9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/TestFpgaDiscoverer.java
@@ -0,0 +1,187 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga;
+
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.fpga.FpgaResourceAllocator;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestFpgaDiscoverer {
+
+  private String getTestParentFolder() {
+    File f = new File("target/temp/" + TestFpgaDiscoverer.class.getName());
+    return f.getAbsolutePath();
+  }
+
+  private void touchFile(File f) throws IOException {
+    new FileOutputStream(f).close();
+  }
+
+  @Before
+  public void before() throws IOException {
+    String folder = getTestParentFolder();
+    File f = new File(folder);
+    FileUtils.deleteDirectory(f);
+    f.mkdirs();
+  }
+
+  @Test
+  public void testLinuxFpgaResourceDiscoverPluginConfig() throws YarnException, IOException {
+    Configuration conf = new Configuration(false);
+    FpgaDiscoverer discoverer = FpgaDiscoverer.getInstance();
+
+    IntelFpgaOpenclPlugin openclPlugin = new IntelFpgaOpenclPlugin();
+    // because FPGA discoverer is a singleton, we use setPlugin to make
+    // FpgaDiscoverer.getInstance().diagnose() work in openclPlugin.initPlugin()
+    discoverer.setResourceHanderPlugin(openclPlugin);
+    openclPlugin.initPlugin(conf);
+    openclPlugin.setShell(mockPuginShell());
+
+    discoverer.initialize(conf);
+    // Case 1. No configuration set for binary
+    Assert.assertEquals("No configuration should return just a single binary name",
+        "aocl", openclPlugin.getPathToExecutable());
+
+    // Case 2. With correct configuration and file exists
+    File fakeBinary = new File(getTestParentFolder() + "/aocl");
+    conf.set(YarnConfiguration.NM_FPGA_PATH_TO_EXEC, getTestParentFolder() + "/aocl");
+    touchFile(fakeBinary);
+    discoverer.initialize(conf);
+    Assert.assertEquals("Correct configuration should return user setting",
+        getTestParentFolder() + "/aocl", openclPlugin.getPathToExecutable());
+
+    // Case 3. With correct configuration but file doesn't exists. Use default
+    fakeBinary.delete();
+    discoverer.initialize(conf);
+    Assert.assertEquals("Correct configuration but file doesn't exists should return just a single binary name",
+        "aocl", openclPlugin.getPathToExecutable());
+
+  }
+
+  @Test
+  public void testDiscoverPluginParser() throws YarnException {
+    String output = "------------------------- acl0 -------------------------\n" +
+        "Vendor: Nallatech ltd\n" +
+        "Phys Dev Name  Status   Information\n" +
+        "aclnalla_pcie0Passed   nalla_pcie (aclnalla_pcie0)\n" +
+        "                       PCIe dev_id = 2494, bus:slot.func = 02:00.00, Gen3 x8\n" +
+        "                       FPGA temperature = 53.1 degrees C.\n" +
+        "                       Total Card Power Usage = 31.7 Watts.\n" +
+        "                       Device Power Usage = 0.0 Watts.\n" +
+        "DIAGNOSTIC_PASSED" +
+        "---------------------------------------------------------\n";
+    output = output +
+        "------------------------- acl1 -------------------------\n" +
+        "Vendor: Nallatech ltd\n" +
+        "Phys Dev Name  Status   Information\n" +
+        "aclnalla_pcie1Passed   nalla_pcie (aclnalla_pcie1)\n" +
+        "                       PCIe dev_id = 2495, bus:slot.func = 03:00.00, Gen3 x8\n" +
+        "                       FPGA temperature = 43.1 degrees C.\n" +
+        "                       Total Card Power Usage = 11.7 Watts.\n" +
+        "                       Device Power Usage = 0.0 Watts.\n" +
+        "DIAGNOSTIC_PASSED" +
+        "---------------------------------------------------------\n";
+    output = output +
+        "------------------------- acl2 -------------------------\n" +
+        "Vendor: Intel(R) Corporation\n" +
+        "\n" +
+        "Phys Dev Name  Status   Information\n" +
+        "\n" +
+        "acla10_ref0   Passed   Arria 10 Reference Platform (acla10_ref0)\n" +
+        "                       PCIe dev_id = 2494, bus:slot.func = 09:00.00, Gen2 x8\n" +
+        "                       FPGA temperature = 50.5781 degrees C.\n" +
+        "\n" +
+        "DIAGNOSTIC_PASSED\n" +
+        "---------------------------------------------------------\n";
+    Configuration conf = new Configuration(false);
+    IntelFpgaOpenclPlugin openclPlugin = new IntelFpgaOpenclPlugin();
+    FpgaDiscoverer.getInstance().setResourceHanderPlugin(openclPlugin);
+
+    openclPlugin.initPlugin(conf);
+    openclPlugin.setShell(mockPuginShell());
+
+    FpgaDiscoverer.getInstance().initialize(conf);
+
+    List<FpgaResourceAllocator.FpgaDevice> list = new LinkedList<>();
+
+    // Case 1. core parsing
+    openclPlugin.parseDiagnoseInfo(output, list);
+    Assert.assertEquals(3, list.size());
+    Assert.assertEquals("IntelOpenCL", list.get(0).getType());
+    Assert.assertEquals("247", list.get(0).getMajor().toString());
+    Assert.assertEquals("0", list.get(0).getMinor().toString());
+    Assert.assertEquals("acl0", list.get(0).getAliasDevName());
+    Assert.assertEquals("aclnalla_pcie0", list.get(0).getDevName());
+    Assert.assertEquals("02:00.00", list.get(0).getBusNum());
+    Assert.assertEquals("53.1 degrees C", list.get(0).getTemperature());
+    Assert.assertEquals("31.7 Watts", list.get(0).getCardPowerUsage());
+
+    Assert.assertEquals("IntelOpenCL", list.get(1).getType());
+    Assert.assertEquals("247", list.get(1).getMajor().toString());
+    Assert.assertEquals("1", list.get(1).getMinor().toString());
+    Assert.assertEquals("acl1", list.get(1).getAliasDevName());
+    Assert.assertEquals("aclnalla_pcie1", list.get(1).getDevName());
+    Assert.assertEquals("03:00.00", list.get(1).getBusNum());
+    Assert.assertEquals("43.1 degrees C", list.get(1).getTemperature());
+    Assert.assertEquals("11.7 Watts", list.get(1).getCardPowerUsage());
+
+    Assert.assertEquals("IntelOpenCL", list.get(2).getType());
+    Assert.assertEquals("246", list.get(2).getMajor().toString());
+    Assert.assertEquals("0", list.get(2).getMinor().toString());
+    Assert.assertEquals("acl2", list.get(2).getAliasDevName());
+    Assert.assertEquals("acla10_ref0", list.get(2).getDevName());
+    Assert.assertEquals("09:00.00", list.get(2).getBusNum());
+    Assert.assertEquals("50.5781 degrees C", list.get(2).getTemperature());
+    Assert.assertEquals("", list.get(2).getCardPowerUsage());
+
+    // Case 2. check alias map
+    Map<String, String> aliasMap = openclPlugin.getAliasMap();
+    Assert.assertEquals("acl0", aliasMap.get("247:0"));
+    Assert.assertEquals("acl1", aliasMap.get("247:1"));
+    Assert.assertEquals("acl2", aliasMap.get("246:0"));
+  }
+
+  private IntelFpgaOpenclPlugin.InnerShellExecutor mockPuginShell() {
+    IntelFpgaOpenclPlugin.InnerShellExecutor shell = mock(IntelFpgaOpenclPlugin.InnerShellExecutor.class);
+    when(shell.runDiagnose(anyString(),anyInt())).thenReturn("");
+    when(shell.getMajorAndMinorNumber("aclnalla_pcie0")).thenReturn("247:0");
+    when(shell.getMajorAndMinorNumber("aclnalla_pcie1")).thenReturn("247:1");
+    when(shell.getMajorAndMinorNumber("acla10_ref0")).thenReturn("246:0");
+    return shell;
+  }
+}


---------------------------------------------------------------------
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: HDFS-11751. DFSZKFailoverController daemon exits with wrong status code. Contributed by Bharat Viswanadham

Posted by ae...@apache.org.
HDFS-11751. DFSZKFailoverController daemon exits with wrong status code. Contributed by Bharat Viswanadham


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

Branch: refs/heads/HDFS-7240
Commit: 6d16a99ae8821c13eec90132e2c63a96fce4b08a
Parents: 99ccca3
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Tue Dec 5 19:53:46 2017 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Tue Dec 5 19:53:46 2017 +0530

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d16a99a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
index a454d73..e0a4f70 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.tools;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.util.ExitUtil.terminate;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
@@ -190,14 +191,13 @@ public class DFSZKFailoverController extends ZKFailoverController {
         new HdfsConfiguration(), args);
     DFSZKFailoverController zkfc = DFSZKFailoverController.create(
         parser.getConfiguration());
-    int retCode = 0;
     try {
-      retCode = zkfc.run(parser.getRemainingArgs());
+      System.exit(zkfc.run(parser.getRemainingArgs()));
     } catch (Throwable t) {
       LOG.fatal("DFSZKFailOverController exiting due to earlier exception "
           + t);
+      terminate(1, t);
     }
-    System.exit(retCode);
   }
 
   @Override


---------------------------------------------------------------------
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: YARN-6507. Add support in NodeManager to isolate FPGA devices with CGroups. (Zhankun Tang via wangda)

Posted by ae...@apache.org.
YARN-6507. Add support in NodeManager to isolate FPGA devices with CGroups. (Zhankun Tang via wangda)

Change-Id: Ic9afd841805f1035423915a0b0add5f3ba96cf9d


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

Branch: refs/heads/HDFS-7240
Commit: 7225ec0ceb49ae8f5588484297a20f07ec047420
Parents: 5304698
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Dec 1 10:50:49 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Dec 1 10:50:49 2017 -0800

----------------------------------------------------------------------
 .../yarn/api/records/ResourceInformation.java   |   5 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |  25 +-
 .../src/main/resources/yarn-default.xml         |  42 +-
 .../linux/privileged/PrivilegedOperation.java   |   1 +
 .../resources/fpga/FpgaResourceAllocator.java   | 413 +++++++++++++++++
 .../resources/fpga/FpgaResourceHandlerImpl.java | 220 +++++++++
 .../resourceplugin/ResourcePluginManager.java   |   8 +-
 .../fpga/AbstractFpgaVendorPlugin.java          |  90 ++++
 .../resourceplugin/fpga/FpgaDiscoverer.java     | 139 ++++++
 .../fpga/FpgaNodeResourceUpdateHandler.java     |  71 +++
 .../resourceplugin/fpga/FpgaResourcePlugin.java | 105 +++++
 .../fpga/IntelFpgaOpenclPlugin.java             | 396 ++++++++++++++++
 .../resources/fpga/TestFpgaResourceHandler.java | 458 +++++++++++++++++++
 .../resourceplugin/fpga/TestFpgaDiscoverer.java | 187 ++++++++
 14 files changed, 2155 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7225ec0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
index 67592cc..a8198d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
@@ -42,6 +42,7 @@ public class ResourceInformation implements Comparable<ResourceInformation> {
   public static final String MEMORY_URI = "memory-mb";
   public static final String VCORES_URI = "vcores";
   public static final String GPU_URI = "yarn.io/gpu";
+  public static final String FPGA_URI = "yarn.io/fpga";
 
   public static final ResourceInformation MEMORY_MB =
       ResourceInformation.newInstance(MEMORY_URI, "Mi");
@@ -49,9 +50,11 @@ public class ResourceInformation implements Comparable<ResourceInformation> {
       ResourceInformation.newInstance(VCORES_URI);
   public static final ResourceInformation GPUS =
       ResourceInformation.newInstance(GPU_URI);
+  public static final ResourceInformation FPGAS =
+          ResourceInformation.newInstance(FPGA_URI);
 
   public static final Map<String, ResourceInformation> MANDATORY_RESOURCES =
-      ImmutableMap.of(MEMORY_URI, MEMORY_MB, VCORES_URI, VCORES, GPU_URI, GPUS);
+      ImmutableMap.of(MEMORY_URI, MEMORY_MB, VCORES_URI, VCORES, GPU_URI, GPUS, FPGA_URI, FPGAS);
 
   /**
    * Get the name for the resource.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7225ec0c/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 c1024ea..831abf5 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
@@ -1514,13 +1514,36 @@ public class YarnConfiguration extends Configuration {
   public static final String DEFAULT_NVIDIA_DOCKER_PLUGIN_V1_ENDPOINT =
       "http://localhost:3476/v1.0/docker/cli";
 
+  /**
+   * Prefix for FPGA configurations. Work in progress: This configuration
+   * parameter may be changed/removed in the future.
+   */
+  @Private
+  public static final String NM_FPGA_RESOURCE_PREFIX =
+          NM_RESOURCE_PLUGINS + ".fpga.";
+
+  @Private
+  public static final String NM_FPGA_ALLOWED_DEVICES =
+          NM_FPGA_RESOURCE_PREFIX + "allowed-fpga-devices";
+
+  @Private
+  public static final String NM_FPGA_PATH_TO_EXEC =
+          NM_FPGA_RESOURCE_PREFIX + "path-to-discovery-executables";
+
+  @Private
+  public static final String NM_FPGA_VENDOR_PLUGIN =
+      NM_FPGA_RESOURCE_PREFIX + "vendor-plugin.class";
+
+  @Private
+  public static final String DEFAULT_NM_FPGA_VENDOR_PLUGIN =
+      "org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga.IntelFpgaOpenclPlugin";
 
   /** NM Webapp address.**/
   public static final String NM_WEBAPP_ADDRESS = NM_PREFIX + "webapp.address";
   public static final int DEFAULT_NM_WEBAPP_PORT = 8042;
   public static final String DEFAULT_NM_WEBAPP_ADDRESS = "0.0.0.0:" +
     DEFAULT_NM_WEBAPP_PORT;
-  
+
   /** NM Webapp https address.**/
   public static final String NM_WEBAPP_HTTPS_ADDRESS = NM_PREFIX
       + "webapp.https.address";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7225ec0c/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 dd9c6bd..2550c42 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
@@ -3512,7 +3512,8 @@
   <property>
     <description>
       Enable additional discovery/isolation of resources on the NodeManager,
-      split by comma. By default, this is empty. Acceptable values: { "yarn-io/gpu" }.
+      split by comma. By default, this is empty.
+      Acceptable values: { "yarn-io/gpu", "yarn-io/fpga"}.
     </description>
     <name>yarn.nodemanager.resource-plugins</name>
     <value></value>
@@ -3559,6 +3560,43 @@
     <value>http://localhost:3476/v1.0/docker/cli</value>
   </property>
 
->>>>>>> theirs
+  <property>
+    <description>
+      Specify one vendor plugin to handle FPGA devices discovery/IP download/configure.
+      Only IntelFpgaOpenclPlugin is supported by default.
+      We only allow one NM configured with one vendor FPGA plugin now since the end user can put the same
+      vendor's cards in one host. And this also simplify our design.
+    </description>
+    <name>yarn.nodemanager.resource-plugins.fpga.vendor-plugin.class</name>
+    <value>org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga.IntelFpgaOpenclPlugin</value>
+  </property>
+
+  <property>
+    <description>
+      When yarn.nodemanager.resource.fpga.allowed-fpga-devices=auto specified,
+      YARN NodeManager needs to run FPGA discovery binary (now only support
+      IntelFpgaOpenclPlugin) to get FPGA information.
+      When value is empty (default), YARN NodeManager will try to locate
+      discovery executable from vendor plugin's preference
+    </description>
+    <name>yarn.nodemanager.resource-plugins.fpga.path-to-discovery-executables</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
+      Specify FPGA devices which can be managed by YARN NodeManager, split by comma
+      Number of FPGA devices will be reported to RM to make scheduling decisions.
+      Set to auto (default) let YARN automatically discover FPGA resource from
+      system.
+
+      Manually specify FPGA devices if admin only want subset of FPGA devices managed by YARN.
+      At present, since we can only configure one major number in c-e.cfg, FPGA device is
+      identified by their minor device number. A common approach to get minor
+      device number of FPGA is using "aocl diagnose" and check uevent with device name.
+    </description>
+    <name>yarn.nodemanager.resource-plugins.fpga.allowed-fpga-devices</name>
+    <value>0,1</value>
+  </property>
 
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7225ec0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperation.java
index db0b225..ad8c22f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/privileged/PrivilegedOperation.java
@@ -52,6 +52,7 @@ public class PrivilegedOperation {
     ADD_PID_TO_CGROUP(""), //no CLI switch supported yet.
     RUN_DOCKER_CMD("--run-docker"),
     GPU("--module-gpu"),
+    FPGA("--module-fpga"),
     LIST_AS_USER(""); //no CLI switch supported yet.
 
     private final String option;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7225ec0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/fpga/FpgaResourceAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/fpga/FpgaResourceAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/fpga/FpgaResourceAllocator.java
new file mode 100644
index 0000000..62dd3c4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/fpga/FpgaResourceAllocator.java
@@ -0,0 +1,413 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.fpga;
+
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandlerException;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga.FpgaDiscoverer;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.*;
+
+import static org.apache.hadoop.yarn.api.records.ResourceInformation.FPGA_URI;
+
+
+/**
+ * This FPGA resource allocator tends to be used by different FPGA vendor's plugin
+ * A "type" parameter is taken into consideration when allocation
+ * */
+public class FpgaResourceAllocator {
+
+  static final Log LOG = LogFactory.getLog(FpgaResourceAllocator.class);
+
+  private List<FpgaDevice> allowedFpgas = new LinkedList<>();
+
+  //key is resource type of FPGA, vendor plugin supported ID
+  private LinkedHashMap<String, List<FpgaDevice>> availableFpga = new LinkedHashMap<>();
+
+  //key is requetor, aka. container ID
+  private LinkedHashMap<String, List<FpgaDevice>> usedFpgaByRequestor = new LinkedHashMap<>();
+
+  private Context nmContext;
+
+  @VisibleForTesting
+  public HashMap<String, List<FpgaDevice>> getAvailableFpga() {
+    return availableFpga;
+  }
+
+  @VisibleForTesting
+  public List<FpgaDevice> getAllowedFpga() {
+    return allowedFpgas;
+  }
+
+  public FpgaResourceAllocator(Context ctx) {
+    this.nmContext = ctx;
+  }
+
+  @VisibleForTesting
+  public int getAvailableFpgaCount() {
+    int count = 0;
+    for (List<FpgaDevice> l : availableFpga.values()) {
+      count += l.size();
+    }
+    return count;
+  }
+
+  @VisibleForTesting
+  public HashMap<String, List<FpgaDevice>> getUsedFpga() {
+    return usedFpgaByRequestor;
+  }
+
+  @VisibleForTesting
+  public int getUsedFpgaCount() {
+    int count = 0;
+    for (List<FpgaDevice> l : usedFpgaByRequestor.values()) {
+      count += l.size();
+    }
+    return count;
+  }
+
+  public static class FpgaAllocation {
+
+    private List<FpgaDevice> allowed = Collections.emptyList();
+
+    private List<FpgaDevice> denied = Collections.emptyList();
+
+    FpgaAllocation(List<FpgaDevice> allowed, List<FpgaDevice> denied) {
+      if (allowed != null) {
+        this.allowed = ImmutableList.copyOf(allowed);
+      }
+      if (denied != null) {
+        this.denied = ImmutableList.copyOf(denied);
+      }
+    }
+
+    public List<FpgaDevice> getAllowed() {
+      return allowed;
+    }
+
+    public List<FpgaDevice> getDenied() {
+      return denied;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append("\nFpgaAllocation\n\tAllowed:\n");
+      for (FpgaDevice device : allowed) {
+        sb.append("\t\t");
+        sb.append(device + "\n");
+      }
+      sb.append("\tDenied\n");
+      for (FpgaDevice device : denied) {
+        sb.append("\t\t");
+        sb.append(device + "\n");
+      }
+      return sb.toString();
+    }
+  }
+
+  public static class FpgaDevice implements Comparable<FpgaDevice>, Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private String type;
+    private Integer major;
+    private Integer minor;
+    // IP file identifier. matrix multiplication for instance
+    private String IPID;
+    // the device name under /dev
+    private String devName;
+    // the alias device name. Intel use acl number acl0 to acl31
+    private String aliasDevName;
+    // lspci output's bus number: 02:00.00 (bus:slot.func)
+    private String busNum;
+    private String temperature;
+    private String cardPowerUsage;
+
+    public String getType() {
+      return type;
+    }
+
+    public Integer getMajor() {
+      return major;
+    }
+
+    public Integer getMinor() {
+      return minor;
+    }
+
+    public String getIPID() {
+      return IPID;
+    }
+
+    public void setIPID(String IPID) {
+      this.IPID = IPID;
+    }
+
+    public String getDevName() {
+      return devName;
+    }
+
+    public void setDevName(String devName) {
+      this.devName = devName;
+    }
+
+    public String getAliasDevName() {
+      return aliasDevName;
+    }
+
+    public void setAliasDevName(String aliasDevName) {
+      this.aliasDevName = aliasDevName;
+    }
+
+    public String getBusNum() {
+      return busNum;
+    }
+
+    public void setBusNum(String busNum) {
+      this.busNum = busNum;
+    }
+
+    public String getTemperature() {
+      return temperature;
+    }
+
+    public String getCardPowerUsage() {
+      return cardPowerUsage;
+    }
+
+    public FpgaDevice(String type, Integer major, Integer minor, String IPID) {
+      this.type = type;
+      this.major = major;
+      this.minor = minor;
+      this.IPID = IPID;
+    }
+
+    public FpgaDevice(String type, Integer major,
+      Integer minor, String IPID, String devName,
+        String aliasDevName, String busNum, String temperature, String cardPowerUsage) {
+      this.type = type;
+      this.major = major;
+      this.minor = minor;
+      this.IPID = IPID;
+      this.devName = devName;
+      this.aliasDevName = aliasDevName;
+      this.busNum = busNum;
+      this.temperature = temperature;
+      this.cardPowerUsage = cardPowerUsage;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (obj == null) {
+        return false;
+      }
+      if (!(obj instanceof FpgaDevice)) {
+        return false;
+      }
+      FpgaDevice other = (FpgaDevice) obj;
+      if (other.getType().equals(this.type) &&
+          other.getMajor().equals(this.major) &&
+          other.getMinor().equals(this.minor)) {
+        return true;
+      }
+      return false;
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((type == null) ? 0 : type.hashCode());
+      result = prime * result + ((major == null) ? 0 : major.hashCode());
+      result = prime * result + ((minor == null) ? 0 : minor.hashCode());
+      return result;
+    }
+
+    @Override
+    public int compareTo(FpgaDevice o) {
+      return 0;
+    }
+
+    @Override
+    public String toString() {
+      return "FPGA Device:(Type: " + this.type + ", Major: " +
+          this.major + ", Minor: " + this.minor + ", IPID: " + this.IPID + ")";
+    }
+  }
+
+  public synchronized void addFpga(String type, List<FpgaDevice> list) {
+    availableFpga.putIfAbsent(type, new LinkedList<>());
+    for (FpgaDevice device : list) {
+      if (!allowedFpgas.contains(device)) {
+        allowedFpgas.add(device);
+        availableFpga.get(type).add(device);
+      }
+    }
+    LOG.info("Add a list of FPGA Devices: " + list);
+  }
+
+  public synchronized void updateFpga(String requestor,
+      FpgaDevice device, String newIPID) {
+    List<FpgaDevice> usedFpgas = usedFpgaByRequestor.get(requestor);
+    int index = findMatchedFpga(usedFpgas, device);
+    if (-1 != index) {
+      usedFpgas.get(index).setIPID(newIPID);
+    } else {
+      LOG.warn("Failed to update FPGA due to unknown reason " +
+          "that no record for this allocated device:" + device);
+    }
+    LOG.info("Update IPID to " + newIPID +
+        " for this allocated device:" + device);
+  }
+
+  private synchronized int findMatchedFpga(List<FpgaDevice> devices, FpgaDevice item) {
+    int i = 0;
+    for (; i < devices.size(); i++) {
+      if (devices.get(i) == item) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  /**
+   * Assign {@link FpgaAllocation} with preferred IPID, if no, with random FPGAs
+   * @param type vendor plugin supported FPGA device type
+   * @param count requested FPGA slot count
+   * @param container container id
+   * @param IPIDPreference allocate slot with this IPID first
+   * @return Instance consists two List of allowed and denied {@link FpgaDevice}
+   * @throws ResourceHandlerException When failed to allocate or write state store
+   * */
+  public synchronized FpgaAllocation assignFpga(String type, long count,
+      Container container, String IPIDPreference) throws ResourceHandlerException {
+    List<FpgaDevice> currentAvailableFpga = availableFpga.get(type);
+    String requestor = container.getContainerId().toString();
+    if (null == currentAvailableFpga) {
+      throw new ResourceHandlerException("No such type of FPGA resource available: " + type);
+    }
+    if (count < 0 || count > currentAvailableFpga.size()) {
+      throw new ResourceHandlerException("Invalid FPGA request count or not enough, requested:" +
+          count + ", available:" + getAvailableFpgaCount());
+    }
+    if (count > 0) {
+      // Allocate devices with matching IP first, then any device is ok
+      List<FpgaDevice> assignedFpgas = new LinkedList<>();
+      int matchIPCount = 0;
+      for (int i = 0; i < currentAvailableFpga.size(); i++) {
+        if ( null != currentAvailableFpga.get(i).getIPID() &&
+            currentAvailableFpga.get(i).getIPID().equalsIgnoreCase(IPIDPreference)) {
+          assignedFpgas.add(currentAvailableFpga.get(i));
+          currentAvailableFpga.remove(i);
+          matchIPCount++;
+        }
+      }
+      int remaining = (int) count - matchIPCount;
+      while (remaining > 0) {
+        assignedFpgas.add(currentAvailableFpga.remove(0));
+        remaining--;
+      }
+
+      // Record in state store if we allocated anything
+      if (!assignedFpgas.isEmpty()) {
+        try {
+          nmContext.getNMStateStore().storeAssignedResources(container,
+              FPGA_URI, new LinkedList<>(assignedFpgas));
+        } catch (IOException e) {
+          // failed, give the allocation back
+          currentAvailableFpga.addAll(assignedFpgas);
+          throw new ResourceHandlerException(e);
+        }
+
+        // update state store success, update internal used FPGAs
+        usedFpgaByRequestor.putIfAbsent(requestor, new LinkedList<>());
+        usedFpgaByRequestor.get(requestor).addAll(assignedFpgas);
+      }
+
+      return new FpgaAllocation(assignedFpgas, currentAvailableFpga);
+    }
+    return new FpgaAllocation(null, allowedFpgas);
+  }
+
+  public synchronized void recoverAssignedFpgas(ContainerId containerId) throws ResourceHandlerException {
+    Container c = nmContext.getContainers().get(containerId);
+    if (null == c) {
+      throw new ResourceHandlerException(
+          "This shouldn't happen, cannot find container with id="
+              + containerId);
+    }
+
+    for (Serializable fpgaDevice :
+        c.getResourceMappings().getAssignedResources(FPGA_URI)) {
+      if (!(fpgaDevice instanceof FpgaDevice)) {
+        throw new ResourceHandlerException(
+            "Trying to recover allocated FPGA devices, however it"
+                + " is not FpgaDevice type, this shouldn't happen");
+      }
+
+      // Make sure it is in allowed FPGA device.
+      if (!allowedFpgas.contains(fpgaDevice)) {
+        throw new ResourceHandlerException("Try to recover FpgaDevice = " + fpgaDevice
+            + " however it is not in allowed device list:" + StringUtils
+            .join(";", allowedFpgas));
+      }
+
+      // Make sure it is not occupied by anybody else
+      Iterator<Map.Entry<String, List<FpgaDevice>>> iterator =
+          getUsedFpga().entrySet().iterator();
+      while (iterator.hasNext()) {
+        if (iterator.next().getValue().contains(fpgaDevice)) {
+          throw new ResourceHandlerException("Try to recover FpgaDevice = " + fpgaDevice
+              + " however it is already assigned to others");
+        }
+      }
+      getUsedFpga().putIfAbsent(containerId.toString(), new LinkedList<>());
+      getUsedFpga().get(containerId.toString()).add((FpgaDevice) fpgaDevice);
+      // remove them from available list
+      getAvailableFpga().get(((FpgaDevice) fpgaDevice).getType()).remove(fpgaDevice);
+    }
+  }
+
+  public synchronized void cleanupAssignFpgas(String requestor) {
+    List<FpgaDevice> usedFpgas = usedFpgaByRequestor.get(requestor);
+    if (usedFpgas != null) {
+      for (FpgaDevice device : usedFpgas) {
+        // Add back to availableFpga
+        availableFpga.get(device.getType()).add(device);
+      }
+      usedFpgaByRequestor.remove(requestor);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7225ec0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/fpga/FpgaResourceHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/fpga/FpgaResourceHandlerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/fpga/FpgaResourceHandlerImpl.java
new file mode 100644
index 0000000..bf3d9b0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/fpga/FpgaResourceHandlerImpl.java
@@ -0,0 +1,220 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.fpga;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationException;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationExecutor;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.CGroupsHandler;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandler;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandlerException;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga.AbstractFpgaVendorPlugin;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga.FpgaDiscoverer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.yarn.api.records.ResourceInformation.FPGA_URI;
+
+@InterfaceStability.Unstable
+@InterfaceAudience.Private
+public class FpgaResourceHandlerImpl implements ResourceHandler {
+
+  static final Log LOG = LogFactory.getLog(FpgaResourceHandlerImpl.class);
+
+  private final String REQUEST_FPGA_IP_ID_KEY = "REQUESTED_FPGA_IP_ID";
+
+  private AbstractFpgaVendorPlugin vendorPlugin;
+
+  private FpgaResourceAllocator allocator;
+
+  private CGroupsHandler cGroupsHandler;
+
+  public static final String EXCLUDED_FPGAS_CLI_OPTION = "--excluded_fpgas";
+  public static final String CONTAINER_ID_CLI_OPTION = "--container_id";
+  private PrivilegedOperationExecutor privilegedOperationExecutor;
+
+  @VisibleForTesting
+  public FpgaResourceHandlerImpl(Context nmContext,
+      CGroupsHandler cGroupsHandler,
+      PrivilegedOperationExecutor privilegedOperationExecutor,
+      AbstractFpgaVendorPlugin plugin) {
+    this.allocator = new FpgaResourceAllocator(nmContext);
+    this.vendorPlugin = plugin;
+    FpgaDiscoverer.getInstance().setResourceHanderPlugin(vendorPlugin);
+    this.cGroupsHandler = cGroupsHandler;
+    this.privilegedOperationExecutor = privilegedOperationExecutor;
+  }
+
+  @VisibleForTesting
+  public FpgaResourceAllocator getFpgaAllocator() {
+    return allocator;
+  }
+
+  public String getRequestedIPID(Container container) {
+    String r= container.getLaunchContext().getEnvironment().
+        get(REQUEST_FPGA_IP_ID_KEY);
+    return r == null ? "" : r;
+  }
+
+  @Override
+  public List<PrivilegedOperation> bootstrap(Configuration configuration) throws ResourceHandlerException {
+    // The plugin should be initilized by FpgaDiscoverer already
+    if (!vendorPlugin.initPlugin(configuration)) {
+      throw new ResourceHandlerException("FPGA plugin initialization failed", null);
+    }
+    LOG.info("FPGA Plugin bootstrap success.");
+    // Get avialable devices minor numbers from toolchain or static configuration
+    List<FpgaResourceAllocator.FpgaDevice> fpgaDeviceList = FpgaDiscoverer.getInstance().discover();
+    allocator.addFpga(vendorPlugin.getFpgaType(), fpgaDeviceList);
+    this.cGroupsHandler.initializeCGroupController(CGroupsHandler.CGroupController.DEVICES);
+    return null;
+  }
+
+  @Override
+  public List<PrivilegedOperation> preStart(Container container) throws ResourceHandlerException {
+    // 1. Get requested FPGA type and count, choose corresponding FPGA plugin(s)
+    // 2. Use allocator.assignFpga(type, count) to get FPGAAllocation
+    // 3. If required, download to ensure IP file exists and configure IP file for all devices
+    List<PrivilegedOperation> ret = new ArrayList<>();
+    String containerIdStr = container.getContainerId().toString();
+    Resource requestedResource = container.getResource();
+
+    // Create device cgroups for the container
+    cGroupsHandler.createCGroup(CGroupsHandler.CGroupController.DEVICES,
+      containerIdStr);
+
+    long deviceCount = requestedResource.getResourceValue(FPGA_URI);
+    LOG.info(containerIdStr + " requested " + deviceCount + " Intel FPGA(s)");
+    String ipFilePath = null;
+    try {
+
+      // allocate even request 0 FPGA because we need to deny all device numbers for this container
+      FpgaResourceAllocator.FpgaAllocation allocation = allocator.assignFpga(
+          vendorPlugin.getFpgaType(), deviceCount,
+          container, getRequestedIPID(container));
+      LOG.info("FpgaAllocation:" + allocation);
+
+      PrivilegedOperation privilegedOperation = new PrivilegedOperation(PrivilegedOperation.OperationType.FPGA,
+          Arrays.asList(CONTAINER_ID_CLI_OPTION, containerIdStr));
+      if (!allocation.getDenied().isEmpty()) {
+        List<Integer> denied = new ArrayList<>();
+        allocation.getDenied().forEach(device -> denied.add(device.getMinor()));
+        privilegedOperation.appendArgs(Arrays.asList(EXCLUDED_FPGAS_CLI_OPTION,
+            StringUtils.join(",", denied)));
+      }
+      privilegedOperationExecutor.executePrivilegedOperation(privilegedOperation, true);
+
+      if (deviceCount > 0) {
+        /**
+         * We only support flashing one IP for all devices now. If user don't set this
+         * environment variable, we assume that user's application can find the IP file by
+         * itself.
+         * Note that the IP downloading and reprogramming in advance in YARN is not necessary because
+         * the OpenCL application may find the IP file and reprogram device on the fly. But YARN do this
+         * for the containers will achieve the quickest reprogram path
+         *
+         * For instance, REQUESTED_FPGA_IP_ID = "matrix_mul" will make all devices
+         * programmed with matrix multiplication IP
+         *
+         * In the future, we may support "matrix_mul:1,gzip:2" format to support different IP
+         * for different devices
+         *
+         * */
+        ipFilePath = vendorPlugin.downloadIP(getRequestedIPID(container), container.getWorkDir(),
+            container.getResourceSet().getLocalizedResources());
+        if (ipFilePath.isEmpty()) {
+          LOG.warn("FPGA plugin failed to download IP but continue, please check the value of environment viable: " +
+              REQUEST_FPGA_IP_ID_KEY + " if you want yarn to help");
+        } else {
+          LOG.info("IP file path:" + ipFilePath);
+          List<FpgaResourceAllocator.FpgaDevice> allowed = allocation.getAllowed();
+          String majorMinorNumber;
+          for (int i = 0; i < allowed.size(); i++) {
+            majorMinorNumber = allowed.get(i).getMajor() + ":" + allowed.get(i).getMinor();
+            String currentIPID = allowed.get(i).getIPID();
+            if (null != currentIPID &&
+                currentIPID.equalsIgnoreCase(getRequestedIPID(container))) {
+              LOG.info("IP already in device \"" + allowed.get(i).getAliasDevName() + "," +
+                  majorMinorNumber + "\", skip reprogramming");
+              continue;
+            }
+            if (vendorPlugin.configureIP(ipFilePath, majorMinorNumber)) {
+              // update the allocator that we update an IP of a device
+              allocator.updateFpga(containerIdStr, allowed.get(i),
+                  getRequestedIPID(container));
+              //TODO: update the node constraint label
+            }
+          }
+        }
+      }
+    } catch (ResourceHandlerException re) {
+      allocator.cleanupAssignFpgas(containerIdStr);
+      cGroupsHandler.deleteCGroup(CGroupsHandler.CGroupController.DEVICES,
+          containerIdStr);
+      throw re;
+    } catch (PrivilegedOperationException e) {
+      allocator.cleanupAssignFpgas(containerIdStr);
+      cGroupsHandler.deleteCGroup(CGroupsHandler.CGroupController.DEVICES, containerIdStr);
+      LOG.warn("Could not update cgroup for container", e);
+      throw new ResourceHandlerException(e);
+    }
+    //isolation operation
+    ret.add(new PrivilegedOperation(
+        PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
+        PrivilegedOperation.CGROUP_ARG_PREFIX
+        + cGroupsHandler.getPathForCGroupTasks(
+        CGroupsHandler.CGroupController.DEVICES, containerIdStr)));
+    return ret;
+  }
+
+  @Override
+  public List<PrivilegedOperation> reacquireContainer(ContainerId containerId) throws ResourceHandlerException {
+    allocator.recoverAssignedFpgas(containerId);
+    return null;
+  }
+
+  @Override
+  public List<PrivilegedOperation> postComplete(ContainerId containerId) throws ResourceHandlerException {
+    allocator.cleanupAssignFpgas(containerId.toString());
+    cGroupsHandler.deleteCGroup(CGroupsHandler.CGroupController.DEVICES,
+        containerId.toString());
+    return null;
+  }
+
+  @Override
+  public List<PrivilegedOperation> teardown() throws ResourceHandlerException {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7225ec0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/ResourcePluginManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/ResourcePluginManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/ResourcePluginManager.java
index 73d6038..12d679b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/ResourcePluginManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/ResourcePluginManager.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga.FpgaResourcePlugin;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.gpu.GpuResourcePlugin;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -33,6 +34,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.hadoop.yarn.api.records.ResourceInformation.FPGA_URI;
 import static org.apache.hadoop.yarn.api.records.ResourceInformation.GPU_URI;
 
 /**
@@ -42,7 +44,7 @@ public class ResourcePluginManager {
   private static final Logger LOG =
       LoggerFactory.getLogger(ResourcePluginManager.class);
   private static final Set<String> SUPPORTED_RESOURCE_PLUGINS = ImmutableSet.of(
-      GPU_URI);
+      GPU_URI, FPGA_URI);
 
   private Map<String, ResourcePlugin> configuredPlugins = Collections.EMPTY_MAP;
 
@@ -77,6 +79,10 @@ public class ResourcePluginManager {
           plugin = new GpuResourcePlugin();
         }
 
+        if (resourceName.equals(FPGA_URI)) {
+          plugin = new FpgaResourcePlugin();
+        }
+
         if (plugin == null) {
           throw new YarnException(
               "This shouldn't happen, plugin=" + resourceName

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7225ec0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/AbstractFpgaVendorPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/AbstractFpgaVendorPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/AbstractFpgaVendorPlugin.java
new file mode 100644
index 0000000..60ea57c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/AbstractFpgaVendorPlugin.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.fpga.FpgaResourceAllocator;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * FPGA plugin interface for vendor to implement. Used by {@link FpgaDiscoverer} and
+ * {@link org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.fpga.FpgaResourceHandlerImpl}
+ * to discover devices/download IP/configure IP
+ * */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface AbstractFpgaVendorPlugin extends Configurable{
+
+  /**
+   * Check vendor's toolchain and required environment
+   * */
+  boolean initPlugin(Configuration conf);
+
+  /**
+   * Diagnose the devices using vendor toolchain but no need to parse device information
+   * */
+  boolean diagnose(int timeout);
+
+  /**
+   * Discover the vendor's FPGA devices with execution time constraint
+   * @param timeout The vendor plugin should return result during this time
+   * @return The result will be added to FPGAResourceAllocator for later scheduling
+   * */
+  List<FpgaResourceAllocator.FpgaDevice> discover(int timeout);
+
+  /**
+   * Since all vendor plugins share a {@link org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.fpga.FpgaResourceAllocator}
+   * which distinguish FPGA devices by type. Vendor plugin must report this.
+   * */
+  String getFpgaType();
+
+  /**
+   * The vendor plugin download required IP files to a required directory.
+   * It should check if the IP file has already been downloaded.
+   * @param id The identifier for IP file. Comes from application, ie. matrix_multi_v1
+   * @param dstDir The plugin should download IP file to this directory
+   * @param localizedResources The container localized resource can be searched for IP file. Key is
+   * localized file path and value is soft link names
+   * @return The absolute path string of IP file
+   * */
+  String downloadIP(String id, String dstDir, Map<Path, List<String>> localizedResources);
+
+  /**
+   * The vendor plugin configure an IP file to a device
+   * @param ipPath The absolute path of the IP file
+   * @param majorMinorNumber The device in format <major:minor>
+   * @return configure device ok or not
+   * */
+  boolean configureIP(String ipPath, String majorMinorNumber);
+
+  @Override
+  void setConf(Configuration conf);
+
+  @Override
+  Configuration getConf();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7225ec0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/FpgaDiscoverer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/FpgaDiscoverer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/FpgaDiscoverer.java
new file mode 100644
index 0000000..8d32a18
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/FpgaDiscoverer.java
@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandlerException;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.fpga.FpgaResourceAllocator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Iterator;
+import java.util.List;
+
+public class FpgaDiscoverer {
+
+  public static final Logger LOG = LoggerFactory.getLogger(
+      FpgaDiscoverer.class);
+
+  private static FpgaDiscoverer instance;
+
+  private Configuration conf = null;
+
+  private AbstractFpgaVendorPlugin plugin = null;
+
+  private List<FpgaResourceAllocator.FpgaDevice> currentFpgaInfo = null;
+
+  // shell command timeout
+  private static final int MAX_EXEC_TIMEOUT_MS = 10 * 1000;
+
+  static {
+    instance = new FpgaDiscoverer();
+  }
+
+  public static FpgaDiscoverer getInstance() {
+    return instance;
+  }
+
+  @VisibleForTesting
+  public synchronized static FpgaDiscoverer setInstance(FpgaDiscoverer newInstance) {
+    instance = newInstance;
+    return instance;
+  }
+
+  @VisibleForTesting
+  public synchronized void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  public List<FpgaResourceAllocator.FpgaDevice> getCurrentFpgaInfo() {
+    return currentFpgaInfo;
+  }
+
+  public synchronized void setResourceHanderPlugin(AbstractFpgaVendorPlugin plugin) {
+    this.plugin = plugin;
+  }
+
+  public synchronized boolean diagnose() {
+    return this.plugin.diagnose(MAX_EXEC_TIMEOUT_MS);
+  }
+
+  public synchronized void initialize(Configuration conf) throws YarnException {
+    this.conf = conf;
+    this.plugin.initPlugin(conf);
+    // Try to diagnose FPGA
+    LOG.info("Trying to diagnose FPGA information ...");
+    if (!diagnose()) {
+      LOG.warn("Failed to pass FPGA devices diagnose");
+    }
+  }
+
+  /**
+   * get avialable devices minor numbers from toolchain or static configuration
+   * */
+  public synchronized List<FpgaResourceAllocator.FpgaDevice> discover() throws ResourceHandlerException {
+    List<FpgaResourceAllocator.FpgaDevice> list;
+    String allowed = this.conf.get(YarnConfiguration.NM_FPGA_ALLOWED_DEVICES);
+    // whatever static or auto discover, we always needs
+    // the vendor plugin to discover. For instance, IntelFpgaOpenclPlugin need to
+    // setup a mapping of <major:minor> to <aliasDevName>
+    list = this.plugin.discover(MAX_EXEC_TIMEOUT_MS);
+    if (0 == list.size()) {
+      throw new ResourceHandlerException("No FPGA devices detected!");
+    }
+    currentFpgaInfo = list;
+    if (allowed.equalsIgnoreCase(
+        YarnConfiguration.AUTOMATICALLY_DISCOVER_GPU_DEVICES)) {
+        return list;
+    } else if (allowed.matches("(\\d,)*\\d")){
+      String[] minors = allowed.split(",");
+      Iterator<FpgaResourceAllocator.FpgaDevice> iterator = list.iterator();
+      // remove the non-configured minor numbers
+      FpgaResourceAllocator.FpgaDevice t;
+      while (iterator.hasNext()) {
+        boolean valid = false;
+        t = iterator.next();
+        for (String minorNumber : minors) {
+          if (t.getMinor().toString().equals(minorNumber)) {
+            valid = true;
+            break;
+          }
+        }
+        if (!valid) {
+          iterator.remove();
+        }
+      }
+      // if the count of user configured is still larger than actual
+      if (list.size() != minors.length) {
+        LOG.warn("We continue although there're mistakes in user's configuration " +
+            YarnConfiguration.NM_FPGA_ALLOWED_DEVICES +
+            "user configured:" + allowed + ", while the real:" + list.toString());
+      }
+    } else {
+      throw new ResourceHandlerException("Invalid value configured for " +
+          YarnConfiguration.NM_FPGA_ALLOWED_DEVICES + ":\"" + allowed + "\"");
+    }
+    return list;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7225ec0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/FpgaNodeResourceUpdateHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/FpgaNodeResourceUpdateHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/FpgaNodeResourceUpdateHandler.java
new file mode 100644
index 0000000..7511d8f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/FpgaNodeResourceUpdateHandler.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga;
+
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.fpga.FpgaResourceAllocator;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.NodeResourceUpdaterPlugin;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.yarn.api.records.ResourceInformation.FPGA_URI;
+
+public class FpgaNodeResourceUpdateHandler extends NodeResourceUpdaterPlugin {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      FpgaNodeResourceUpdateHandler.class);
+
+  @Override
+  public void updateConfiguredResource(Resource res) throws YarnException {
+    LOG.info("Initializing configured FPGA resources for the NodeManager.");
+    List<FpgaResourceAllocator.FpgaDevice> list = FpgaDiscoverer.getInstance().getCurrentFpgaInfo();
+    List<Integer> minors = new LinkedList<>();
+    for (FpgaResourceAllocator.FpgaDevice device : list) {
+      minors.add(device.getMinor());
+    }
+    if (minors.isEmpty()) {
+      LOG.info("Didn't find any usable FPGAs on the NodeManager.");
+      return;
+    }
+    long count = minors.size();
+
+    Map<String, ResourceInformation> configuredResourceTypes =
+        ResourceUtils.getResourceTypes();
+    if (!configuredResourceTypes.containsKey(FPGA_URI)) {
+      throw new YarnException("Wrong configurations, found " + count +
+          " usable FPGAs, however " + FPGA_URI
+          + " resource-type is not configured inside"
+          + " resource-types.xml, please configure it to enable FPGA feature or"
+          + " remove " + FPGA_URI + " from "
+          + YarnConfiguration.NM_RESOURCE_PLUGINS);
+    }
+
+    res.setResourceValue(FPGA_URI, count);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7225ec0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/FpgaResourcePlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/FpgaResourcePlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/FpgaResourcePlugin.java
new file mode 100644
index 0000000..44d093e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/FpgaResourcePlugin.java
@@ -0,0 +1,105 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationExecutor;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.CGroupsHandler;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandler;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.fpga.FpgaResourceHandlerImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.DockerCommandPlugin;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.NodeResourceUpdaterPlugin;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePlugin;
+import org.apache.hadoop.yarn.server.nodemanager.webapp.dao.NMResourceInfo;
+
+public class FpgaResourcePlugin implements ResourcePlugin {
+  private static final Log LOG = LogFactory.getLog(FpgaResourcePlugin.class);
+
+  private ResourceHandler fpgaResourceHandler = null;
+
+  private AbstractFpgaVendorPlugin vendorPlugin = null;
+  private FpgaNodeResourceUpdateHandler fpgaNodeResourceUpdateHandler = null;
+
+  private AbstractFpgaVendorPlugin createFpgaVendorPlugin(Configuration conf) {
+    String vendorPluginClass = conf.get(YarnConfiguration.NM_FPGA_VENDOR_PLUGIN,
+        YarnConfiguration.DEFAULT_NM_FPGA_VENDOR_PLUGIN);
+    LOG.info("Using FPGA vendor plugin: " + vendorPluginClass);
+    try {
+      Class<?> schedulerClazz = Class.forName(vendorPluginClass);
+      if (AbstractFpgaVendorPlugin.class.isAssignableFrom(schedulerClazz)) {
+        return (AbstractFpgaVendorPlugin) ReflectionUtils.newInstance(schedulerClazz,
+            conf);
+      } else {
+        throw new YarnRuntimeException("Class: " + vendorPluginClass
+            + " not instance of " + AbstractFpgaVendorPlugin.class.getCanonicalName());
+      }
+    } catch (ClassNotFoundException e) {
+      throw new YarnRuntimeException("Could not instantiate FPGA vendor plugin: "
+          + vendorPluginClass, e);
+    }
+  }
+
+  @Override
+  public void initialize(Context context) throws YarnException {
+    // Get vendor plugin from configuration
+    this.vendorPlugin = createFpgaVendorPlugin(context.getConf());
+    FpgaDiscoverer.getInstance().setResourceHanderPlugin(vendorPlugin);
+    FpgaDiscoverer.getInstance().initialize(context.getConf());
+    fpgaNodeResourceUpdateHandler = new FpgaNodeResourceUpdateHandler();
+  }
+
+  @Override
+  public ResourceHandler createResourceHandler(
+      Context nmContext, CGroupsHandler cGroupsHandler,
+      PrivilegedOperationExecutor privilegedOperationExecutor) {
+    if (fpgaResourceHandler == null) {
+      fpgaResourceHandler = new FpgaResourceHandlerImpl(nmContext,
+          cGroupsHandler, privilegedOperationExecutor, vendorPlugin);
+    }
+    return fpgaResourceHandler;
+  }
+
+  @Override
+  public NodeResourceUpdaterPlugin getNodeResourceHandlerInstance() {
+    return fpgaNodeResourceUpdateHandler;
+  }
+
+  @Override
+  public void cleanup() throws YarnException {
+
+  }
+
+  @Override
+  public DockerCommandPlugin getDockerCommandPluginInstance() {
+    return null;
+  }
+
+  @Override
+  public NMResourceInfo getNMResourceInfo() throws YarnException {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7225ec0c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/IntelFpgaOpenclPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/IntelFpgaOpenclPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/IntelFpgaOpenclPlugin.java
new file mode 100644
index 0000000..f2e82b8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/IntelFpgaOpenclPlugin.java
@@ -0,0 +1,396 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.fpga.FpgaResourceAllocator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.*;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Intel FPGA for OpenCL plugin.
+ * The key points are:
+ * 1. It uses Intel's toolchain "aocl" to discover devices/reprogram IP to the device
+ *    before container launch to achieve a quickest reprogramming path
+ * 2. It avoids reprogramming by maintaining a mapping of device to FPGA IP ID
+ * 3. It assume IP file is distributed to container directory
+ */
+public class IntelFpgaOpenclPlugin implements AbstractFpgaVendorPlugin {
+  public static final Logger LOG = LoggerFactory.getLogger(
+      IntelFpgaOpenclPlugin.class);
+
+  private boolean initialized = false;
+  private Configuration conf;
+  private InnerShellExecutor shell;
+
+  protected static final String DEFAULT_BINARY_NAME = "aocl";
+
+  protected static final String ALTERAOCLSDKROOT_NAME = "ALTERAOCLSDKROOT";
+
+  private String pathToExecutable = null;
+
+  // a mapping of major:minor number to acl0-31
+  private Map<String, String> aliasMap;
+
+  public IntelFpgaOpenclPlugin() {
+    this.shell = new InnerShellExecutor();
+  }
+
+  public String getDefaultBinaryName() {
+    return DEFAULT_BINARY_NAME;
+  }
+
+  public String getDefaultPathToExecutable() {
+    return System.getenv(ALTERAOCLSDKROOT_NAME);
+  }
+
+  public static String getDefaultPathEnvName() {
+    return ALTERAOCLSDKROOT_NAME;
+  }
+
+  @VisibleForTesting
+  public String getPathToExecutable() {
+    return pathToExecutable;
+  }
+
+  public void setPathToExecutable(String pathToExecutable) {
+    this.pathToExecutable = pathToExecutable;
+  }
+
+  @VisibleForTesting
+  public void setShell(InnerShellExecutor shell) {
+    this.shell = shell;
+  }
+
+  public Map<String, String> getAliasMap() {
+    return aliasMap;
+  }
+
+  /**
+   * Check the Intel FPGA for OpenCL toolchain
+   * */
+  @Override
+  public boolean initPlugin(Configuration conf) {
+    this.aliasMap = new HashMap<>();
+    if (this.initialized) {
+      return true;
+    }
+    // Find the proper toolchain, mainly aocl
+    String pluginDefaultBinaryName = getDefaultBinaryName();
+    String pathToExecutable = conf.get(YarnConfiguration.NM_FPGA_PATH_TO_EXEC,
+        "");
+    if (pathToExecutable.isEmpty()) {
+      pathToExecutable = pluginDefaultBinaryName;
+    }
+    // Validate file existence
+    File binaryPath = new File(pathToExecutable);
+    if (!binaryPath.exists()) {
+      // When binary not exist, fail
+      LOG.warn("Failed to find FPGA discoverer executable configured in " +
+          YarnConfiguration.NM_FPGA_PATH_TO_EXEC +
+          ", please check! Try default path");
+      pathToExecutable = pluginDefaultBinaryName;
+      // Try to find in plugin's preferred path
+      String pluginDefaultPreferredPath = getDefaultPathToExecutable();
+      if (null == pluginDefaultPreferredPath) {
+        LOG.warn("Failed to find FPGA discoverer executable from system environment " +
+            getDefaultPathEnvName()+
+            ", please check your environment!");
+      } else {
+        binaryPath = new File(pluginDefaultPreferredPath + "/bin", pluginDefaultBinaryName);
+        if (binaryPath.exists()) {
+          pathToExecutable = pluginDefaultPreferredPath;
+        } else {
+          pathToExecutable = pluginDefaultBinaryName;
+          LOG.warn("Failed to find FPGA discoverer executable in " +
+              pluginDefaultPreferredPath + ", file doesn't exists! Use default binary" + pathToExecutable);
+        }
+      }
+    }
+    setPathToExecutable(pathToExecutable);
+    if (!diagnose(10*1000)) {
+      LOG.warn("Intel FPGA for OpenCL diagnose failed!");
+      this.initialized = false;
+    } else {
+      this.initialized = true;
+    }
+    return this.initialized;
+  }
+
+  @Override
+  public List<FpgaResourceAllocator.FpgaDevice> discover(int timeout) {
+    List<FpgaResourceAllocator.FpgaDevice> list = new LinkedList<>();
+    String output;
+    output = getDiagnoseInfo(timeout);
+    if (null == output) {
+      return list;
+    }
+    parseDiagnoseInfo(output, list);
+    return list;
+  }
+
+  public static class InnerShellExecutor {
+
+    // ls /dev/<devName>
+    // return a string in format <major:minor>
+    public String getMajorAndMinorNumber(String devName) {
+      String output = null;
+      Shell.ShellCommandExecutor shexec = new Shell.ShellCommandExecutor(
+          new String[]{"stat", "-c", "%t:%T", "/dev/" + devName});
+      try {
+        LOG.debug("Get FPGA major-minor numbers from /dev/" + devName);
+        shexec.execute();
+        String[] strs = shexec.getOutput().trim().split(":");
+        LOG.debug("stat output:" + shexec.getOutput());
+        output = Integer.parseInt(strs[0], 16) + ":" + Integer.parseInt(strs[1], 16);
+      } catch (IOException e) {
+        String msg =
+            "Failed to get major-minor number from reading /dev/" + devName;
+        LOG.warn(msg);
+        LOG.debug("Command output:" + shexec.getOutput() + ", exit code:" +
+            shexec.getExitCode());
+      }
+      return output;
+    }
+
+    public String runDiagnose(String binary, int timeout) {
+      String output = null;
+      Shell.ShellCommandExecutor shexec = new Shell.ShellCommandExecutor(
+          new String[]{binary, "diagnose"});
+      try {
+        shexec.execute();
+      } catch (IOException e) {
+        // aocl diagnose exit code is 1 even it success.
+        // we ignore it because we only wants the output
+        String msg =
+            "Failed to execute " + binary + " diagnose, exception message:" + e
+                .getMessage() +", output:" + output + ", continue ...";
+        LOG.warn(msg);
+        LOG.debug(shexec.getOutput());
+      }
+      return shexec.getOutput();
+    }
+
+  }
+
+  /**
+   * One real sample output of Intel FPGA SDK 17.0's "aocl diagnose" is as below:
+   * "
+   * aocl diagnose: Running diagnose from /home/fpga/intelFPGA_pro/17.0/hld/board/nalla_pcie/linux64/libexec
+   *
+   * ------------------------- acl0 -------------------------
+   * Vendor: Nallatech ltd
+   *
+   * Phys Dev Name  Status   Information
+   *
+   * aclnalla_pcie0Passed   nalla_pcie (aclnalla_pcie0)
+   *                        PCIe dev_id = 2494, bus:slot.func = 02:00.00, Gen3 x8
+   *                        FPGA temperature = 54.4 degrees C.
+   *                        Total Card Power Usage = 31.7 Watts.
+   *                        Device Power Usage = 0.0 Watts.
+   *
+   * DIAGNOSTIC_PASSED
+   * ---------------------------------------------------------
+   * "
+   *
+   * While per Intel's guide, the output(should be outdated or prior SDK version's) is as below:
+   *
+   * "
+   * aocl diagnose: Running diagnostic from ALTERAOCLSDKROOT/board/<board_name>/
+   * <platform>/libexec
+   * Verified that the kernel mode driver is installed on the host machine.
+   * Using board package from vendor: <board_vendor_name>
+   * Querying information for all supported devices that are installed on the host
+   * machine ...
+   *
+   * device_name Status Information
+   *
+   * acl0 Passed <descriptive_board_name>
+   *             PCIe dev_id = <device_ID>, bus:slot.func = 02:00.00,
+   *               at Gen 2 with 8 lanes.
+   *             FPGA temperature=43.0 degrees C.
+   * acl1 Passed <descriptive_board_name>
+   *             PCIe dev_id = <device_ID>, bus:slot.func = 03:00.00,
+   *               at Gen 2 with 8 lanes.
+   *             FPGA temperature = 35.0 degrees C.
+   *
+   * Found 2 active device(s) installed on the host machine, to perform a full
+   * diagnostic on a specific device, please run aocl diagnose <device_name>
+   *
+   * DIAGNOSTIC_PASSED
+   * "
+   * But this method only support the first output
+   * */
+  public void parseDiagnoseInfo(String output, List<FpgaResourceAllocator.FpgaDevice> list) {
+    if (output.contains("DIAGNOSTIC_PASSED")) {
+      Matcher headerStartMatcher = Pattern.compile("acl[0-31]").matcher(output);
+      Matcher headerEndMatcher = Pattern.compile("(?i)DIAGNOSTIC_PASSED").matcher(output);
+      int sectionStartIndex;
+      int sectionEndIndex;
+      String aliasName;
+      while (headerStartMatcher.find()) {
+        sectionStartIndex = headerStartMatcher.end();
+        String section = null;
+        aliasName = headerStartMatcher.group();
+        while (headerEndMatcher.find(sectionStartIndex)) {
+          sectionEndIndex = headerEndMatcher.start();
+          section = output.substring(sectionStartIndex, sectionEndIndex);
+          break;
+        }
+        if (null == section) {
+          LOG.warn("Unsupported diagnose output");
+          return;
+        }
+        // devName, \(.*\)
+        // busNum, bus:slot.func\s=\s.*,
+        // FPGA temperature\s=\s.*
+        // Total\sCard\sPower\sUsage\s=\s.*
+        String[] fieldRegexes = new String[]{"\\(.*\\)\n", "(?i)bus:slot.func\\s=\\s.*,",
+            "(?i)FPGA temperature\\s=\\s.*", "(?i)Total\\sCard\\sPower\\sUsage\\s=\\s.*"};
+        String[] fields = new String[4];
+        String tempFieldValue;
+        for (int i = 0; i < fieldRegexes.length; i++) {
+          Matcher fieldMatcher = Pattern.compile(fieldRegexes[i]).matcher(section);
+          if (!fieldMatcher.find()) {
+            LOG.warn("Couldn't find " + fieldRegexes[i] + " pattern");
+            fields[i] = "";
+            continue;
+          }
+          tempFieldValue = fieldMatcher.group().trim();
+          if (i == 0) {
+            // special case for Device name
+            fields[i] = tempFieldValue.substring(1, tempFieldValue.length() - 1);
+          } else {
+            String ss = tempFieldValue.split("=")[1].trim();
+            fields[i] = ss.substring(0, ss.length() - 1);
+          }
+        }
+        String majorMinorNumber = this.shell.getMajorAndMinorNumber(fields[0]);
+        if (null != majorMinorNumber) {
+          String[] mmn = majorMinorNumber.split(":");
+          this.aliasMap.put(majorMinorNumber, aliasName);
+          list.add(new FpgaResourceAllocator.FpgaDevice(getFpgaType(),
+              Integer.parseInt(mmn[0]),
+              Integer.parseInt(mmn[1]), null,
+              fields[0], aliasName, fields[1], fields[2], fields[3]));
+        }
+      }// end while
+    }// end if
+  }
+
+  public String getDiagnoseInfo(int timeout) {
+    return this.shell.runDiagnose(this.pathToExecutable,timeout);
+  }
+
+  @Override
+  public boolean diagnose(int timeout) {
+    String output = getDiagnoseInfo(timeout);
+    if (null != output && output.contains("DIAGNOSTIC_PASSED")) {
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * this is actually the opencl platform type
+   * */
+  @Override
+  public String getFpgaType() {
+    return "IntelOpenCL";
+  }
+
+  @Override
+  public String downloadIP(String id, String dstDir, Map<Path, List<String>> localizedResources) {
+    // Assume .aocx IP file is distributed by DS to local dir
+    String r = "";
+    Path path;
+    LOG.info("Got environment: " + id + ", search IP file in localized resources");
+    if (null == id || id.isEmpty()) {
+      LOG.warn("IP_ID environment is empty, skip downloading");
+      return r;
+    }
+    if (localizedResources != null) {
+      for (Map.Entry<Path, List<String>> resourceEntry :
+          localizedResources.entrySet()) {
+        path = resourceEntry.getKey();
+        LOG.debug("Check:" + path.toUri().toString());
+        if (path.getName().toLowerCase().contains(id.toLowerCase()) && path.getName().endsWith(".aocx")) {
+          r = path.toUri().toString();
+          LOG.debug("Found: " + r);
+          break;
+        }
+      }
+    } else {
+      LOG.warn("Localized resource is null!");
+    }
+    return r;
+  }
+
+  /**
+   * Program one device.
+   * It's ok for the offline "aocl program" failed because the application will always invoke API to program
+   * The reason we do offline reprogramming is to make the application's program process faster
+   * @param ipPath the absolute path to the aocx IP file
+   * @param majorMinorNumber major:minor string
+   * @return True or False
+   * */
+  @Override
+  public boolean configureIP(String ipPath, String majorMinorNumber) {
+    // perform offline program the IP to get a quickest reprogramming sequence
+    // we need a mapping of "major:minor" to "acl0" to issue command "aocl program <acl0> <ipPath>"
+    Shell.ShellCommandExecutor shexec;
+    String aclName;
+    aclName = this.aliasMap.get(majorMinorNumber);
+    shexec = new Shell.ShellCommandExecutor(
+        new String[]{this.pathToExecutable, "program", aclName, ipPath});
+    try {
+      shexec.execute();
+      if (0 == shexec.getExitCode()) {
+        LOG.debug(shexec.getOutput());
+        LOG.info("Intel aocl program " + ipPath + " to " + aclName + " successfully");
+      } else {
+        return false;
+      }
+    } catch (IOException e) {
+      LOG.error("Intel aocl program " + ipPath + " to " + aclName + " failed!");
+      e.printStackTrace();
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return this.conf;
+  }
+}


---------------------------------------------------------------------
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: YARN-7381. Enable the configuration: yarn.nodemanager.log-container-debug-info.enabled by default in yarn-default.xml. (Xuan Gong via wangda)

Posted by ae...@apache.org.
YARN-7381. Enable the configuration: yarn.nodemanager.log-container-debug-info.enabled by default in yarn-default.xml. (Xuan Gong via wangda)

Change-Id: I1ed58dafad5cc276eea5c0b0813cf04f57d73a87


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

Branch: refs/heads/HDFS-7240
Commit: 0311cf05358cd75388f48f048c44fba52ec90f00
Parents: 6555af8
Author: Wangda Tan <wa...@apache.org>
Authored: Tue Dec 5 13:09:49 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Tue Dec 5 13:09:49 2017 -0800

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0311cf05/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 831abf5..e57f988 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
@@ -142,7 +142,7 @@ public class YarnConfiguration extends Configuration {
   public static final String NM_LOG_CONTAINER_DEBUG_INFO =
       YarnConfiguration.NM_PREFIX + "log-container-debug-info.enabled";
 
-  public static final boolean DEFAULT_NM_LOG_CONTAINER_DEBUG_INFO = false;
+  public static final boolean DEFAULT_NM_LOG_CONTAINER_DEBUG_INFO = true;
 
   ////////////////////////////////
   // IPC Configs

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0311cf05/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 2550c42..192f62e 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
@@ -1282,7 +1282,7 @@
     slowness in launching containers.
     </description>
     <name>yarn.nodemanager.log-container-debug-info.enabled</name>
-    <value>false</value>
+    <value>true</value>
   </property>
 
   <property>


---------------------------------------------------------------------
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: YARN-7495. Improve robustness of the AggregatedLogDeletionService. Contributed by Jonathan Eagles

Posted by ae...@apache.org.
YARN-7495. Improve robustness of the AggregatedLogDeletionService. Contributed by Jonathan Eagles


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

Branch: refs/heads/HDFS-7240
Commit: 5cfaee2e6db8b2ac55708de0968ff5539ee3bd76
Parents: 75a3ab8
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Nov 30 12:39:18 2017 -0600
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Nov 30 12:39:18 2017 -0600

----------------------------------------------------------------------
 .../AggregatedLogDeletionService.java           | 90 ++++++++++++--------
 .../TestAggregatedLogDeletionService.java       | 68 +++++++++++++++
 2 files changed, 122 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5cfaee2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java
index a80f9d7..562bd2c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java
@@ -85,49 +85,67 @@ public class AggregatedLogDeletionService extends AbstractService {
             deleteOldLogDirsFrom(userDirPath, cutoffMillis, fs, rmClient);
           }
         }
-      } catch (IOException e) {
-        logIOException("Error reading root log dir this deletion " +
-        		"attempt is being aborted", e);
+      } catch (Throwable t) {
+        logException("Error reading root log dir this deletion " +
+            "attempt is being aborted", t);
       }
       LOG.info("aggregated log deletion finished.");
     }
     
     private static void deleteOldLogDirsFrom(Path dir, long cutoffMillis, 
         FileSystem fs, ApplicationClientProtocol rmClient) {
+      FileStatus[] appDirs;
       try {
-        for(FileStatus appDir : fs.listStatus(dir)) {
-          if(appDir.isDirectory() && 
-              appDir.getModificationTime() < cutoffMillis) {
-            boolean appTerminated =
-                isApplicationTerminated(ApplicationId.fromString(appDir
-                  .getPath().getName()), rmClient);
-            if(appTerminated && shouldDeleteLogDir(appDir, cutoffMillis, fs)) {
-              try {
-                LOG.info("Deleting aggregated logs in "+appDir.getPath());
-                fs.delete(appDir.getPath(), true);
-              } catch (IOException e) {
-                logIOException("Could not delete "+appDir.getPath(), e);
-              }
-            } else if (!appTerminated){
-              try {
-                for(FileStatus node: fs.listStatus(appDir.getPath())) {
-                  if(node.getModificationTime() < cutoffMillis) {
-                    try {
-                      fs.delete(node.getPath(), true);
-                    } catch (IOException ex) {
-                      logIOException("Could not delete "+appDir.getPath(), ex);
-                    }
-                  }
+        appDirs = fs.listStatus(dir);
+      } catch (IOException e) {
+        logException("Could not read the contents of " + dir, e);
+        return;
+      }
+      for (FileStatus appDir : appDirs) {
+        deleteAppDirLogs(cutoffMillis, fs, rmClient, appDir);
+      }
+    }
+
+    private static void deleteAppDirLogs(long cutoffMillis, FileSystem fs,
+                                         ApplicationClientProtocol rmClient,
+                                         FileStatus appDir) {
+      try {
+        if (appDir.isDirectory() &&
+            appDir.getModificationTime() < cutoffMillis) {
+          ApplicationId appId = ApplicationId.fromString(
+              appDir.getPath().getName());
+          boolean appTerminated = isApplicationTerminated(appId, rmClient);
+          if (!appTerminated) {
+            // Application is still running
+            FileStatus[] logFiles;
+            try {
+              logFiles = fs.listStatus(appDir.getPath());
+            } catch (IOException e) {
+              logException("Error reading the contents of "
+                  + appDir.getPath(), e);
+              return;
+            }
+            for (FileStatus node : logFiles) {
+              if (node.getModificationTime() < cutoffMillis) {
+                try {
+                  fs.delete(node.getPath(), true);
+                } catch (IOException ex) {
+                  logException("Could not delete " + appDir.getPath(), ex);
                 }
-              } catch(IOException e) {
-                logIOException(
-                  "Error reading the contents of " + appDir.getPath(), e);
               }
             }
+          } else if (shouldDeleteLogDir(appDir, cutoffMillis, fs)) {
+            // Application is no longer running
+            try {
+              LOG.info("Deleting aggregated logs in " + appDir.getPath());
+              fs.delete(appDir.getPath(), true);
+            } catch (IOException e) {
+              logException("Could not delete " + appDir.getPath(), e);
+            }
           }
         }
-      } catch (IOException e) {
-        logIOException("Could not read the contents of " + dir, e);
+      } catch (Exception e) {
+        logException("Could not delete " + appDir.getPath(), e);
       }
     }
 
@@ -142,7 +160,7 @@ public class AggregatedLogDeletionService extends AbstractService {
           }
         }
       } catch(IOException e) {
-        logIOException("Error reading the contents of " + dir.getPath(), e);
+        logException("Error reading the contents of " + dir.getPath(), e);
         shouldDelete = false;
       }
       return shouldDelete;
@@ -172,14 +190,14 @@ public class AggregatedLogDeletionService extends AbstractService {
     }
   }
   
-  private static void logIOException(String comment, IOException e) {
-    if(e instanceof AccessControlException) {
-      String message = e.getMessage();
+  private static void logException(String comment, Throwable t) {
+    if(t instanceof AccessControlException) {
+      String message = t.getMessage();
       //TODO fix this after HADOOP-8661
       message = message.split("\n")[0];
       LOG.warn(comment + " " + message);
     } else {
-      LOG.error(comment, e);
+      LOG.error(comment, t);
     }
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5cfaee2e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java
index 026996e..4e2d302 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java
@@ -385,6 +385,74 @@ public class TestAggregatedLogDeletionService {
     deletionSvc.stop();
   }
 
+  @Test
+  public void testRobustLogDeletion() throws Exception {
+    final long RETENTION_SECS = 10 * 24 * 3600;
+
+    String root = "mockfs://foo/";
+    String remoteRootLogDir = root+"tmp/logs";
+    String suffix = "logs";
+    Configuration conf = new Configuration();
+    conf.setClass("fs.mockfs.impl", MockFileSystem.class,
+        FileSystem.class);
+    conf.set(YarnConfiguration.LOG_AGGREGATION_ENABLED, "true");
+    conf.set(YarnConfiguration.LOG_AGGREGATION_RETAIN_SECONDS, "864000");
+    conf.set(YarnConfiguration.LOG_AGGREGATION_RETAIN_CHECK_INTERVAL_SECONDS,
+        "1");
+    conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteRootLogDir);
+    conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR_SUFFIX, suffix);
+
+    // prevent us from picking up the same mockfs instance from another test
+    FileSystem.closeAll();
+    Path rootPath = new Path(root);
+    FileSystem rootFs = rootPath.getFileSystem(conf);
+    FileSystem mockFs = ((FilterFileSystem)rootFs).getRawFileSystem();
+
+    Path remoteRootLogPath = new Path(remoteRootLogDir);
+
+    Path userDir = new Path(remoteRootLogPath, "me");
+    FileStatus userDirStatus = new FileStatus(0, true, 0, 0, 0, userDir);
+
+    when(mockFs.listStatus(remoteRootLogPath)).thenReturn(
+        new FileStatus[]{userDirStatus});
+
+    Path userLogDir = new Path(userDir, suffix);
+    ApplicationId appId1 =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    Path app1Dir = new Path(userLogDir, appId1.toString());
+    FileStatus app1DirStatus = new FileStatus(0, true, 0, 0, 0, app1Dir);
+    ApplicationId appId2 =
+        ApplicationId.newInstance(System.currentTimeMillis(), 2);
+    Path app2Dir = new Path(userLogDir, "application_a");
+    FileStatus app2DirStatus = new FileStatus(0, true, 0, 0, 0, app2Dir);
+    ApplicationId appId3 =
+        ApplicationId.newInstance(System.currentTimeMillis(), 3);
+    Path app3Dir = new Path(userLogDir, appId3.toString());
+    FileStatus app3DirStatus = new FileStatus(0, true, 0, 0, 0, app3Dir);
+
+    when(mockFs.listStatus(userLogDir)).thenReturn(
+        new FileStatus[]{app1DirStatus, app2DirStatus, app3DirStatus});
+
+    when(mockFs.listStatus(app1Dir)).thenThrow(
+        new RuntimeException("Should Be Caught and Logged"));
+    Path app3Log3 = new Path(app3Dir, "host1");
+    FileStatus app3Log3Status = new FileStatus(10, false, 1, 1, 0, app3Log3);
+    when(mockFs.listStatus(app3Dir)).thenReturn(
+        new FileStatus[]{app3Log3Status});
+
+    final List<ApplicationId> finishedApplications =
+        Collections.unmodifiableList(Arrays.asList(appId1, appId3));
+
+    ApplicationClientProtocol rmClient =
+        createMockRMClient(finishedApplications, null);
+    AggregatedLogDeletionService.LogDeletionTask deletionTask =
+        new AggregatedLogDeletionService.LogDeletionTask(conf,
+            RETENTION_SECS,
+            rmClient);
+    deletionTask.run();
+    verify(mockFs).delete(app3Dir, true);
+  }
+
   static class MockFileSystem extends FilterFileSystem {
     MockFileSystem() {
       super(mock(FileSystem.class));


---------------------------------------------------------------------
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-7589. TestPBImplRecords fails with NullPointerException. Contributed by Daniel Templeton

Posted by ae...@apache.org.
YARN-7589. TestPBImplRecords fails with NullPointerException. Contributed by Daniel Templeton


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

Branch: refs/heads/HDFS-7240
Commit: 25df5054216a6a76d09d9c49984f8075ebc6a197
Parents: c83fe44
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Dec 1 15:37:36 2017 -0600
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Dec 1 15:37:36 2017 -0600

----------------------------------------------------------------------
 .../org/apache/hadoop/yarn/api/records/Resource.java   |  9 ++++++---
 .../hadoop/yarn/util/resource/ResourceUtils.java       | 13 +++++++++----
 2 files changed, 15 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/25df5054/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
index b32955b..304a963 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
@@ -102,9 +102,12 @@ public abstract class Resource implements Comparable<Resource> {
   @Stable
   public static Resource newInstance(long memory, int vCores,
       Map<String, Long> others) {
-    ResourceInformation[] info = ResourceUtils.createResourceTypesArray(others);
-
-    return new LightWeightResource(memory, vCores, info);
+    if (others != null) {
+      return new LightWeightResource(memory, vCores,
+          ResourceUtils.createResourceTypesArray(others));
+    } else {
+      return newInstance(memory, vCores);
+    }
   }
 
   @InterfaceAudience.Private

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25df5054/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
index 3c6ca98..76ae061 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
@@ -313,15 +313,13 @@ public class ResourceUtils {
   }
 
   public static ResourceInformation[] getResourceTypesArray() {
-    initializeResourceTypesIfNeeded(null,
-        YarnConfiguration.RESOURCE_TYPES_CONFIGURATION_FILE);
+    initializeResourceTypesIfNeeded();
     return resourceTypesArray;
   }
 
   public static int getNumberOfKnownResourceTypes() {
     if (numKnownResourceTypes < 0) {
-      initializeResourceTypesIfNeeded(null,
-          YarnConfiguration.RESOURCE_TYPES_CONFIGURATION_FILE);
+      initializeResourceTypesIfNeeded();
     }
     return numKnownResourceTypes;
   }
@@ -332,6 +330,11 @@ public class ResourceUtils {
         YarnConfiguration.RESOURCE_TYPES_CONFIGURATION_FILE);
   }
 
+  private static void initializeResourceTypesIfNeeded() {
+    initializeResourceTypesIfNeeded(null,
+        YarnConfiguration.RESOURCE_TYPES_CONFIGURATION_FILE);
+  }
+
   private static void initializeResourceTypesIfNeeded(Configuration conf,
       String resourceFile) {
     if (!initializedResources) {
@@ -641,6 +644,8 @@ public class ResourceUtils {
    */
   public static ResourceInformation[] createResourceTypesArray(Map<String,
       Long> res) {
+    initializeResourceTypesIfNeeded();
+
     ResourceInformation[] info = new ResourceInformation[resourceTypes.size()];
 
     for (Entry<String, Integer> entry : RESOURCE_NAME_TO_INDEX.entrySet()) {


---------------------------------------------------------------------
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: HADOOP-14985. Remove subversion related code from VersionInfoMojo.java. Contributed by Ajay Kumar.

Posted by ae...@apache.org.
HADOOP-14985. Remove subversion related code from VersionInfoMojo.java. Contributed by Ajay Kumar.


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

Branch: refs/heads/HDFS-7240
Commit: 9f1bdafedb60f83598819c6a682f659d6e168eb0
Parents: e00c7f7
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Dec 5 14:30:46 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Dec 5 14:30:46 2017 +0900

----------------------------------------------------------------------
 .../plugin/versioninfo/VersionInfoMojo.java     | 85 +++-----------------
 1 file changed, 12 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f1bdafe/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java
----------------------------------------------------------------------
diff --git a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java
index f6faea0..ca970c6 100644
--- a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java
+++ b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java
@@ -72,10 +72,7 @@ public class VersionInfoMojo extends AbstractMojo {
   @Parameter(defaultValue="git")
   private String gitCommand;
 
-  @Parameter(defaultValue="svn")
-  private String svnCommand;
-
-  private enum SCM {NONE, SVN, GIT}
+  private enum SCM {NONE, GIT}
 
   @Override
   public void execute() throws MojoExecutionException {
@@ -104,7 +101,7 @@ public class VersionInfoMojo extends AbstractMojo {
   private List<String> scmOut;
 
   /**
-   * Determines which SCM is in use (Subversion, git, or none) and captures
+   * Determines which SCM is in use (git or none) and captures
    * output of the SCM command for later parsing.
    * 
    * @return SCM in use for this build
@@ -114,27 +111,24 @@ public class VersionInfoMojo extends AbstractMojo {
     Exec exec = new Exec(this);
     SCM scm = SCM.NONE;
     scmOut = new ArrayList<String>();
-    int ret = exec.run(Arrays.asList(svnCommand, "info"), scmOut);
+    int ret;
+    ret = exec.run(Arrays.asList(gitCommand, "branch"), scmOut);
     if (ret == 0) {
-      scm = SCM.SVN;
-    } else {
-      ret = exec.run(Arrays.asList(gitCommand, "branch"), scmOut);
-      if (ret == 0) {
-        ret = exec.run(Arrays.asList(gitCommand, "remote", "-v"), scmOut);
+      ret = exec.run(Arrays.asList(gitCommand, "remote", "-v"), scmOut);
+      if (ret != 0) {
+        scm = SCM.NONE;
+        scmOut = null;
+      } else {
+        ret = exec.run(Arrays.asList(gitCommand, "log", "-n", "1"), scmOut);
         if (ret != 0) {
           scm = SCM.NONE;
           scmOut = null;
         } else {
-          ret = exec.run(Arrays.asList(gitCommand, "log", "-n", "1"), scmOut);
-          if (ret != 0) {
-            scm = SCM.NONE;
-            scmOut = null;
-          } else {
-            scm = SCM.GIT;
-          }
+          scm = SCM.GIT;
         }
       }
     }
+
     if (scmOut != null) {
       getLog().debug(scmOut.toString());
     }
@@ -143,35 +137,6 @@ public class VersionInfoMojo extends AbstractMojo {
   }
 
   /**
-   * Return URI and branch of Subversion repository.
-   * 
-   * @param str String Subversion info output containing URI and branch
-   * @return String[] containing URI and branch
-   */
-  private String[] getSvnUriInfo(String str) {
-    String[] res = new String[]{"Unknown", "Unknown"};
-    String path = str;
-    int index = path.indexOf("trunk");
-    if (index > -1) {
-      res[0] = path.substring(0, index - 1);
-      res[1] = "trunk";
-    } else {
-      index = path.indexOf("branches");
-      if (index > -1) {
-        res[0] = path.substring(0, index - 1);
-        int branchIndex = index + "branches".length() + 1;
-        index = path.indexOf('/', branchIndex);
-        if (index > -1) {
-          res[1] = path.substring(branchIndex, index);
-        } else {
-          res[1] = path.substring(branchIndex);
-        }
-      }
-    }
-    return res;
-  }
-
-  /**
    * Parses SCM output and returns URI of SCM.
    * 
    * @param scm SCM in use for this build
@@ -180,15 +145,6 @@ public class VersionInfoMojo extends AbstractMojo {
   private String getSCMUri(SCM scm) {
     String uri = "Unknown";
     switch (scm) {
-      case SVN:
-        for (String s : scmOut) {
-          if (s.startsWith("URL:")) {
-            uri = s.substring(4).trim();
-            uri = getSvnUriInfo(uri)[0];
-            break;
-          }
-        }
-        break;
       case GIT:
         for (String s : scmOut) {
           if (s.startsWith("origin") && s.endsWith("(fetch)")) {
@@ -211,14 +167,6 @@ public class VersionInfoMojo extends AbstractMojo {
   private String getSCMCommit(SCM scm) {
     String commit = "Unknown";
     switch (scm) {
-      case SVN:
-        for (String s : scmOut) {
-          if (s.startsWith("Revision:")) {
-            commit = s.substring("Revision:".length());
-            break;
-          }
-        }
-        break;
       case GIT:
         for (String s : scmOut) {
           if (s.startsWith("commit")) {
@@ -240,15 +188,6 @@ public class VersionInfoMojo extends AbstractMojo {
   private String getSCMBranch(SCM scm) {
     String branch = "Unknown";
     switch (scm) {
-      case SVN:
-        for (String s : scmOut) {
-          if (s.startsWith("URL:")) {
-            branch = s.substring(4).trim();
-            branch = getSvnUriInfo(branch)[1];
-            break;
-          }
-        }
-        break;
       case GIT:
         for (String s : scmOut) {
           if (s.startsWith("*")) {


---------------------------------------------------------------------
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-7455. quote_and_append_arg can overflow buffer. Contributed by Jim Brennan

Posted by ae...@apache.org.
YARN-7455. quote_and_append_arg can overflow buffer. Contributed by Jim Brennan


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

Branch: refs/heads/HDFS-7240
Commit: 60f95fb719f00a718b484c36a823ec5aa8b099f4
Parents: 25df505
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Dec 1 15:47:01 2017 -0600
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Dec 1 15:47:01 2017 -0600

----------------------------------------------------------------------
 .../main/native/container-executor/impl/util.c  |  25 +--
 .../main/native/container-executor/impl/util.h  |   3 +-
 .../native/container-executor/test/test_util.cc | 160 +++++++++++++++++--
 3 files changed, 161 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/60f95fb7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.c
index a9539cf..eea3e10 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.c
@@ -21,6 +21,7 @@
 #include <string.h>
 #include <ctype.h>
 #include <regex.h>
+#include <stdio.h>
 
 char** split_delimiter(char *value, const char *delim) {
   char **return_values = NULL;
@@ -176,17 +177,19 @@ char* escape_single_quote(const char *str) {
 
 void quote_and_append_arg(char **str, size_t *size, const char* param, const char *arg) {
   char *tmp = escape_single_quote(arg);
-  int alloc_block = 1024;
-  strcat(*str, param);
-  strcat(*str, "'");
-  if (strlen(*str) + strlen(tmp) > *size) {
-    *size = (strlen(*str) + strlen(tmp) + alloc_block) * sizeof(char);
-    *str = (char *) realloc(*str, *size);
-    if (*str == NULL) {
-      exit(OUT_OF_MEMORY);
-    }
+  const char *append_format = "%s'%s' ";
+  size_t append_size = snprintf(NULL, 0, append_format, param, tmp);
+  append_size += 1;   // for the terminating NUL
+  size_t len_str = strlen(*str);
+  size_t new_size = len_str + append_size;
+  if (new_size > *size) {
+      *size = new_size + QUOTE_AND_APPEND_ARG_GROWTH;
+      *str = (char *) realloc(*str, *size);
+      if (*str == NULL) {
+          exit(OUT_OF_MEMORY);
+      }
   }
-  strcat(*str, tmp);
-  strcat(*str, "' ");
+  char *cur_ptr = *str + len_str;
+  sprintf(cur_ptr, append_format, param, tmp);
   free(tmp);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60f95fb7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h
index 8758d90..ed9fba8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h
@@ -141,12 +141,13 @@ char* escape_single_quote(const char *str);
 
 /**
  * Helper function to quote the argument to a parameter and then append it to the provided string.
- * @param str Buffer to which the param='arg' string must be appended
+ * @param str Buffer to which the param'arg' string must be appended
  * @param size Size of the buffer
  * @param param Param name
  * @param arg Argument to be quoted
  */
 void quote_and_append_arg(char **str, size_t *size, const char* param, const char *arg);
+#define QUOTE_AND_APPEND_ARG_GROWTH (1024) // how much to increase str buffer by if needed
 
 /**
  * Helper function to allocate and clear a block of memory. It'll call exit if the allocation fails.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60f95fb7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_util.cc
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_util.cc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_util.cc
index b96dea1..8cdbf2f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_util.cc
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_util.cc
@@ -149,25 +149,155 @@ namespace ContainerExecutor {
     }
   }
 
+  /**
+   * Internal function for testing quote_and_append_arg()
+   */
+  void test_quote_and_append_arg_function_internal(char **str, size_t *size, const char* param, const char *arg, const char *expected_result) {
+    const size_t alloc_block_size = QUOTE_AND_APPEND_ARG_GROWTH;
+    size_t orig_size = *size;
+    size_t expected_size = strlen(expected_result) + 1;
+    if (expected_size > orig_size) {
+      expected_size += alloc_block_size;
+    } else {
+      expected_size = orig_size; // fits in original string
+    }
+    quote_and_append_arg(str, size, param, arg);
+    ASSERT_STREQ(*str, expected_result);
+    ASSERT_EQ(*size, expected_size);
+    return;
+  }
+
   TEST_F(TestUtil, test_quote_and_append_arg) {
+    size_t str_real_size = 32;
+
+    // Simple test - size = 32, result = 16
+    size_t str_size = str_real_size;
+    char *str = (char *) malloc(str_real_size);
+    memset(str, 0, str_real_size);
+    strcpy(str, "ssss");
+    const char *param = "pppp";
+    const char *arg = "aaaa";
+    const char *expected_result = "sssspppp'aaaa' ";
+    test_quote_and_append_arg_function_internal(&str, &str_size, param, arg, expected_result);
+    free(str);
+
+    // Original test - size = 32, result = 19
+    str_size = str_real_size;
+    str = (char *) malloc(str_real_size);
+    memset(str, 0, str_real_size);
+    param = "param=";
+    arg = "argument1";
+    expected_result = "param='argument1' ";
+    test_quote_and_append_arg_function_internal(&str, &str_size, param, arg, expected_result);
+    free(str);
+
+    // Original test - size = 32 and result = 21
+    str_size = str_real_size;
+    str = (char *) malloc(str_real_size);
+    memset(str, 0, str_real_size);
+    param = "param=";
+    arg = "ab'cd";
+    expected_result = "param='ab'\"'\"'cd' "; // 21 characters
+    test_quote_and_append_arg_function_internal(&str, &str_size, param, arg, expected_result);
+    free(str);
+
+    // Lie about size of buffer so we don't crash from an actual buffer overflow
+    // Original Test - Size = 4 and result = 19
+    str_size = 4;
+    str = (char *) malloc(str_real_size);
+    memset(str, 0, str_real_size);
+    param = "param=";
+    arg = "argument1";
+    expected_result = "param='argument1' "; // 19 characters
+    test_quote_and_append_arg_function_internal(&str, &str_size, param, arg, expected_result);
+    free(str);
+
+    // Size = 8 and result = 7
+    str_size = 8;
+    str = (char *) malloc(str_real_size);
+    memset(str, 0, str_real_size);
+    strcpy(str, "s");
+    param = "p";
+    arg = "a";
+    expected_result = "sp'a' "; // 7 characters
+    test_quote_and_append_arg_function_internal(&str, &str_size, param, arg, expected_result);
+    free(str);
+
+    // Size = 8 and result = 7
+    str_size = 8;
+    str = (char *) malloc(str_real_size);
+    memset(str, 0, str_real_size);
+    strcpy(str, "s");
+    param = "p";
+    arg = "a";
+    expected_result = "sp'a' "; // 7 characters
+    test_quote_and_append_arg_function_internal(&str, &str_size, param, arg, expected_result);
+    free(str);
+
+    // Size = 8 and result = 8
+    str_size = 8;
+    str = (char *) malloc(str_real_size);
+    memset(str, 0, str_real_size);
+    strcpy(str, "ss");
+    param = "p";
+    arg = "a";
+    expected_result = "ssp'a' "; // 8 characters
+    test_quote_and_append_arg_function_internal(&str, &str_size, param, arg, expected_result);
+    free(str);
+
+    // size = 8, result = 9 (should grow buffer)
+    str_size = 8;
+    str = (char *) malloc(str_real_size);
+    memset(str, 0, str_real_size);
+    strcpy(str, "ss");
+    param = "pp";
+    arg = "a";
+    expected_result = "sspp'a' "; // 9 characters
+    test_quote_and_append_arg_function_internal(&str, &str_size, param, arg, expected_result);
+    free(str);
 
-    char *tmp = static_cast<char *>(malloc(4096));
-    size_t tmp_size = 4096;
+    // size = 8, result = 10 (should grow buffer)
+    str_size = 8;
+    str = (char *) malloc(str_real_size);
+    memset(str, 0, str_real_size);
+    strcpy(str, "ss");
+    param = "pp";
+    arg = "aa";
+    expected_result = "sspp'aa' "; // 10 characters
+    test_quote_and_append_arg_function_internal(&str, &str_size, param, arg, expected_result);
+    free(str);
 
-    memset(tmp, 0, tmp_size);
-    quote_and_append_arg(&tmp, &tmp_size, "param=", "argument1");
-    ASSERT_STREQ("param='argument1' ", tmp);
+    // size = 8, result = 11 (should grow buffer)
+    str_size = 8;
+    str = (char *) malloc(str_real_size);
+    memset(str, 0, str_real_size);
+    strcpy(str, "sss");
+    param = "pp";
+    arg = "aa";
+    expected_result = "ssspp'aa' "; // 11 characters
+    test_quote_and_append_arg_function_internal(&str, &str_size, param, arg, expected_result);
+    free(str);
 
-    memset(tmp, 0, tmp_size);
-    quote_and_append_arg(&tmp, &tmp_size, "param=", "ab'cd");
-    ASSERT_STREQ("param='ab'\"'\"'cd' ", tmp);
-    free(tmp);
+    // One with quotes - size = 32, result = 17
+    str_size = 32;
+    str = (char *) malloc(str_real_size);
+    memset(str, 0, str_real_size);
+    strcpy(str, "s");
+    param = "p";
+    arg = "'a'";
+    expected_result = "sp''\"'\"'a'\"'\"'' "; // 17 characters
+    test_quote_and_append_arg_function_internal(&str, &str_size, param, arg, expected_result);
+    free(str);
 
-    tmp = static_cast<char *>(malloc(4));
-    tmp_size = 4;
-    memset(tmp, 0, tmp_size);
-    quote_and_append_arg(&tmp, &tmp_size, "param=", "argument1");
-    ASSERT_STREQ("param='argument1' ", tmp);
-    ASSERT_EQ(1040, tmp_size);
+    // One with quotes - size = 16, result = 17
+    str_size = 16;
+    str = (char *) malloc(str_real_size);
+    memset(str, 0, str_real_size);
+    strcpy(str, "s");
+    param = "p";
+    arg = "'a'";
+    expected_result = "sp''\"'\"'a'\"'\"'' "; // 17 characters
+    test_quote_and_append_arg_function_internal(&str, &str_size, param, arg, expected_result);
+    free(str);
   }
 }


---------------------------------------------------------------------
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: Merge branch 'trunk' into HDFS-7240

Posted by ae...@apache.org.
Merge branch 'trunk' into HDFS-7240

 Conflicts:
	hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml


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

Branch: refs/heads/HDFS-7240
Commit: 5f16cdbd28ef4484253fc8f789c42d90e26c3543
Parents: 15cce449 40b0045e
Author: Anu Engineer <ae...@apache.org>
Authored: Wed Dec 6 16:27:04 2017 -0800
Committer: Anu Engineer <ae...@apache.org>
Committed: Wed Dec 6 16:27:04 2017 -0800

----------------------------------------------------------------------
 dev-support/bin/create-release                  |  71 ++-
 .../hadoop-common/src/main/bin/hadoop           |   7 +-
 .../crypto/key/KeyProviderTokenIssuer.java      |  36 ++
 .../apache/hadoop/crypto/key/package-info.java  |  18 +
 .../java/org/apache/hadoop/fs/FileStatus.java   |  61 ++-
 .../java/org/apache/hadoop/fs/FileSystem.java   |  15 +
 .../org/apache/hadoop/fs/LocatedFileStatus.java |  42 +-
 .../apache/hadoop/fs/RawLocalFileSystem.java    |  77 ++-
 .../hadoop/fs/permission/FsPermission.java      |  34 ++
 .../apache/hadoop/fs/protocolPB/PBHelper.java   |  11 +-
 .../org/apache/hadoop/io/nativeio/NativeIO.java |  34 ++
 .../org/apache/hadoop/io/nativeio/NativeIO.c    |  85 +++
 .../src/site/markdown/Compatibility.md          |  29 +-
 .../apache/hadoop/fs/TestFilterFileSystem.java  |   1 +
 .../org/apache/hadoop/fs/TestHarFileSystem.java |   1 +
 .../fs/TestRawLocalFileSystemContract.java      |  87 ++-
 .../hadoop/fs/permission/TestFsPermission.java  |  31 ++
 .../apache/hadoop/io/nativeio/TestNativeIO.java | 144 ++++-
 .../apache/hadoop/test/GenericTestUtils.java    |  10 +-
 .../java/org/apache/hadoop/test/StatUtils.java  | 126 +++++
 .../src/test/scripts/hadoop_shell_execname.bats |  36 ++
 .../dev-support/findbugsExcludeFile.xml         |  31 ++
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 133 +----
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  83 +--
 .../hadoop/hdfs/DistributedFileSystem.java      |  75 ++-
 .../org/apache/hadoop/hdfs/HdfsKMSUtil.java     | 190 +++++++
 .../hdfs/client/HdfsClientConfigKeys.java       |   4 +
 .../impl/SnapshotDiffReportGenerator.java       | 262 +++++++++
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  29 +
 .../hadoop/hdfs/protocol/HdfsFileStatus.java    | 530 ++++++++++---------
 .../hdfs/protocol/HdfsLocatedFileStatus.java    | 208 ++++++--
 .../hdfs/protocol/HdfsNamedFileStatus.java      | 180 +++++++
 .../hadoop/hdfs/protocol/LocatedBlocks.java     |   3 +-
 .../protocol/SnapshotDiffReportListing.java     | 160 ++++++
 .../hadoop/hdfs/protocol/package-info.java      |  18 +
 .../ClientNamenodeProtocolTranslatorPB.java     |  24 +
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  | 184 ++++++-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  37 +-
 .../src/main/proto/ClientNamenodeProtocol.proto |  12 +
 .../src/main/proto/hdfs.proto                   |  26 +
 .../protocol/TestHdfsFileStatusMethods.java     | 106 ++++
 .../hadoop/fs/http/client/HttpFSFileSystem.java |   5 +-
 .../dev-support/findbugsExcludeFile.xml         |   7 -
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |   1 +
 .../hadoop-hdfs/src/main/bin/hdfs               |   9 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   5 +
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |  44 +-
 ...tNamenodeProtocolServerSideTranslatorPB.java |  22 +
 .../qjournal/client/QuorumJournalManager.java   |   6 +
 .../server/blockmanagement/BlockManager.java    |  51 +-
 .../BlockReconstructionWork.java                |  14 +-
 .../blockmanagement/ErasureCodingWork.java      |   4 +-
 .../blockmanagement/PendingRecoveryBlocks.java  | 143 +++++
 .../server/blockmanagement/ReplicationWork.java |   4 +-
 .../resolver/ActiveNamenodeResolver.java        |   7 +-
 .../resolver/MembershipNamenodeResolver.java    |   3 +-
 .../federation/resolver/MountTableResolver.java |   5 +-
 .../resolver/NamenodeStatusReport.java          |   8 +-
 .../federation/router/ConnectionManager.java    |   4 +-
 .../federation/router/ConnectionPool.java       |   4 +-
 .../federation/router/FederationUtil.java       |   2 +-
 .../router/NamenodeHeartbeatService.java        |   6 +-
 .../hdfs/server/federation/router/Router.java   |   6 +-
 .../federation/router/RouterRpcClient.java      |  22 +-
 .../federation/router/RouterRpcServer.java      |  13 +-
 .../federation/store/CachedRecordStore.java     |   7 +-
 .../server/federation/store/RecordStore.java    |   2 +-
 .../store/driver/StateStoreDriver.java          |   9 +-
 .../driver/impl/StateStoreFileBaseImpl.java     |   6 +-
 .../driver/impl/StateStoreFileSystemImpl.java   |   4 +-
 .../driver/impl/StateStoreZooKeeperImpl.java    |   5 +-
 .../store/impl/MembershipStoreImpl.java         |   5 +-
 .../apache/hadoop/hdfs/server/mover/Mover.java  |   2 +-
 .../hdfs/server/namenode/FSDirSnapshotOp.java   |  24 +
 .../server/namenode/FSDirStatAndListingOp.java  |  47 +-
 .../hdfs/server/namenode/FSNamesystem.java      | 117 +++-
 .../hadoop/hdfs/server/namenode/INode.java      |  14 +
 .../hdfs/server/namenode/NameNodeRpcServer.java |  13 +
 .../snapshot/DirectorySnapshottableFeature.java | 136 ++++-
 .../snapshot/SnapshotDiffListingInfo.java       | 207 ++++++++
 .../namenode/snapshot/SnapshotManager.java      |  28 +
 .../hdfs/tools/DFSZKFailoverController.java     |   6 +-
 .../src/main/resources/hdfs-default.xml         |  21 +
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |  10 +
 .../hadoop/hdfs/TestBlockStoragePolicy.java     |   2 +-
 .../apache/hadoop/hdfs/TestEncryptionZones.java |  40 +-
 .../hadoop/hdfs/TestEncryptionZonesWithKMS.java |  23 +
 .../apache/hadoop/hdfs/TestFileChecksum.java    |   9 +-
 .../apache/hadoop/hdfs/TestFileCreation.java    | 103 ++++
 .../hdfs/TestFileStatusSerialization.java       |   8 +-
 .../TestPendingRecoveryBlocks.java              |  87 +++
 .../hdfs/server/datanode/TestBlockRecovery.java | 107 ++++
 .../hdfs/server/mover/TestStorageMover.java     |   2 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |  41 ++
 .../namenode/ha/TestPipelinesFailover.java      |   4 +-
 .../namenode/ha/TestStandbyInProgressTail.java  |  19 +
 .../snapshot/TestSnapshotDiffReport.java        | 116 ++++
 .../src/test/scripts/hadoop_shell_execname.bats |  36 ++
 hadoop-mapreduce-project/bin/mapred             |   8 +-
 .../hadoop/mapred/TaskAttemptListenerImpl.java  |  69 ++-
 .../job/event/TaskAttemptStatusUpdateEvent.java |  12 +-
 .../v2/app/job/impl/TaskAttemptImpl.java        |  20 +-
 .../mapred/TestTaskAttemptListenerImpl.java     | 315 +++++++----
 .../mapreduce/v2/app/TestFetchFailure.java      |   3 +-
 .../mapreduce/v2/app/TestMRClientService.java   |   4 +-
 .../v2/TestSpeculativeExecutionWithMRApp.java   |  13 +-
 .../hadoop-mapreduce-client-uploader/pom.xml    |  67 +++
 .../hadoop/mapred/uploader/DefaultJars.java     |  46 ++
 .../mapred/uploader/FrameworkUploader.java      | 384 ++++++++++++++
 .../mapred/uploader/UploaderException.java      |  36 ++
 .../hadoop/mapred/uploader/package-info.java    |  28 +
 .../mapred/uploader/TestFrameworkUploader.java  | 315 +++++++++++
 .../hadoop-mapreduce-client/pom.xml             |   1 +
 .../plugin/versioninfo/VersionInfoMojo.java     |  85 +--
 hadoop-project/pom.xml                          |   2 +-
 .../s3a/BlockingThreadPoolExecutorService.java  |   7 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |   5 +
 .../hadoop/fs/s3a/S3AInstrumentation.java       | 107 +++-
 .../fs/s3a/SemaphoredDelegatingExecutor.java    |  15 +-
 .../src/site/markdown/tools/hadoop-aws/index.md |  38 +-
 .../tools/hadoop-aws/troubleshooting_s3a.md     | 140 ++++-
 .../ITestBlockingThreadPoolExecutorService.java |   2 +
 .../apache/hadoop/fs/s3a/ITestS3AMetrics.java   |  51 ++
 hadoop-yarn-project/hadoop-yarn/bin/yarn        |   3 +-
 .../hadoop/yarn/api/records/Resource.java       |  27 +-
 .../yarn/api/records/ResourceInformation.java   |   5 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |  27 +-
 .../yarn/util/resource/ResourceUtils.java       |  42 +-
 .../distributedshell/ApplicationMaster.java     |  22 +-
 .../applications/distributedshell/Client.java   |  19 +
 .../distributedshell/TestDistributedShell.java  |  56 ++
 .../hadoop/yarn/service/webapp/ApiServer.java   |   1 +
 ...RN-Simplified-V1-API-Layer-For-Services.yaml |  16 +
 .../dev-support/findbugs-exclude.xml            |   5 +-
 .../yarn/service/ClientAMPolicyProvider.java    |  39 ++
 .../yarn/service/ClientAMSecurityInfo.java      |  62 +++
 .../hadoop/yarn/service/ClientAMService.java    |   9 +
 .../hadoop/yarn/service/ServiceContext.java     |   8 +
 .../hadoop/yarn/service/ServiceMaster.java      | 140 ++++-
 .../hadoop/yarn/service/ServiceScheduler.java   |  30 +-
 .../service/api/records/KerberosPrincipal.java  | 146 +++++
 .../yarn/service/api/records/Service.java       |  23 +
 .../yarn/service/client/ServiceClient.java      | 174 ++++--
 .../yarn/service/component/Component.java       |   2 +-
 .../yarn/service/conf/YarnServiceConf.java      |   7 -
 .../yarn/service/conf/YarnServiceConstants.java |   3 +
 .../containerlaunch/AbstractLauncher.java       |  39 +-
 .../containerlaunch/ContainerLaunchService.java |  10 +-
 .../containerlaunch/CredentialUtils.java        | 319 -----------
 .../hadoop/yarn/service/package-info.java       |  24 +
 .../yarn/service/provider/ProviderUtils.java    |  53 +-
 .../yarn/service/utils/ServiceApiUtil.java      |  15 +
 .../hadoop/yarn/service/utils/ServiceUtils.java |  31 +-
 .../org.apache.hadoop.security.SecurityInfo     |  14 +
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  |  41 +-
 .../AggregatedLogDeletionService.java           |  90 ++--
 .../client/YARNDelegationTokenIdentifier.java   |  11 +
 .../src/main/resources/yarn-default.xml         |  44 +-
 .../TestAggregatedLogDeletionService.java       |  68 +++
 .../yarn/security/TestYARNTokenIdentifier.java  | 115 ++--
 .../client/api/RegistryOperationsFactory.java   |  21 +
 .../registry/client/impl/zk/CuratorService.java |   8 +-
 .../client/impl/zk/RegistrySecurity.java        |  96 +++-
 .../hadoop/registry/server/dns/RegistryDNS.java |   4 +
 .../RMRegistryOperationsService.java            | 246 ---------
 .../services/DeleteCompletionCallback.java      |   3 +-
 .../hadoop/registry/AbstractRegistryTest.java   |  15 +-
 .../integration/TestRegistryRMOperations.java   | 369 -------------
 .../secure/TestSecureRMRegistryOperations.java  | 348 ------------
 .../records/OpportunisticContainersStatus.java  |  19 +
 .../pb/OpportunisticContainersStatusPBImpl.java |  13 +
 .../main/proto/yarn_server_common_protos.proto  |   1 +
 .../linux/privileged/PrivilegedOperation.java   |   1 +
 .../resources/fpga/FpgaResourceAllocator.java   | 413 +++++++++++++++
 .../resources/fpga/FpgaResourceHandlerImpl.java | 220 ++++++++
 .../runtime/DockerLinuxContainerRuntime.java    |  63 ++-
 .../runtime/docker/DockerVolumeCommand.java     |  29 +-
 .../resourceplugin/ResourcePluginManager.java   |   8 +-
 .../fpga/AbstractFpgaVendorPlugin.java          |  90 ++++
 .../resourceplugin/fpga/FpgaDiscoverer.java     | 139 +++++
 .../fpga/FpgaNodeResourceUpdateHandler.java     |  71 +++
 .../resourceplugin/fpga/FpgaResourcePlugin.java | 105 ++++
 .../fpga/IntelFpgaOpenclPlugin.java             | 396 ++++++++++++++
 .../gpu/NvidiaDockerV1CommandPlugin.java        |   2 +-
 .../scheduler/ContainerScheduler.java           |  12 +
 .../nodemanager/webapp/AllContainersPage.java   |   6 +-
 .../nodemanager/webapp/NMWebServices.java       |   4 +-
 .../server/nodemanager/webapp/NodePage.java     |   2 +
 .../server/nodemanager/webapp/dao/NodeInfo.java |   9 +
 .../main/native/container-executor/impl/util.c  |  25 +-
 .../main/native/container-executor/impl/util.h  |   3 +-
 .../container-executor/impl/utils/docker-util.c | 106 ++--
 .../native/container-executor/test/test_util.cc | 160 +++++-
 .../test/utils/test_docker_util.cc              |   5 +-
 .../resources/fpga/TestFpgaResourceHandler.java | 458 ++++++++++++++++
 .../runtime/TestDockerContainerRuntime.java     | 170 +++++-
 .../resourceplugin/fpga/TestFpgaDiscoverer.java | 187 +++++++
 .../nodemanager/webapp/TestNMWebServices.java   |  26 +-
 .../server/resourcemanager/AdminService.java    |  21 +-
 .../server/resourcemanager/RMAppManager.java    |  60 ++-
 .../server/resourcemanager/ResourceManager.java |  31 +-
 .../monitor/SchedulingMonitor.java              |   3 +-
 .../monitor/SchedulingMonitorManager.java       | 184 +++++++
 .../placement/PlacementManager.java             |   5 +-
 .../recovery/FileSystemRMStateStore.java        |   4 +-
 .../recovery/LeveldbRMStateStore.java           |   5 +-
 .../recovery/RMStateStoreUtils.java             |  69 +++
 .../recovery/ZKRMStateStore.java                |   3 +-
 .../RMDelegationTokenIdentifierData.java        |   8 +
 .../server/resourcemanager/rmapp/RMApp.java     |   8 +
 .../server/resourcemanager/rmapp/RMAppImpl.java |  80 +--
 .../rmcontainer/RMContainer.java                |   5 +-
 .../rmcontainer/RMContainerImpl.java            |  20 +-
 .../scheduler/AbstractYarnScheduler.java        |  32 +-
 .../scheduler/AppSchedulingInfo.java            |  66 ++-
 .../scheduler/ClusterNodeTracker.java           |  79 ++-
 .../scheduler/ContainerUpdateContext.java       |  11 +-
 .../scheduler/SchedulerApplicationAttempt.java  |  15 +-
 .../scheduler/capacity/CapacityScheduler.java   |   6 +
 .../allocator/RegularContainerAllocator.java    |   3 +-
 .../scheduler/common/ContainerRequest.java      |  54 ++
 .../scheduler/common/fica/FiCaSchedulerApp.java |  15 +-
 .../distributed/NodeQueueLoadMonitor.java       |  22 +-
 .../scheduler/fair/FSAppAttempt.java            |   5 +-
 .../scheduler/fair/FairScheduler.java           |   6 +
 .../scheduler/fifo/FifoAppAttempt.java          |   5 +-
 .../scheduler/fifo/FifoScheduler.java           |   6 +
 .../placement/AppPlacementAllocator.java        |  19 +-
 .../LocalityAppPlacementAllocator.java          |  32 +-
 .../placement/PendingAskUpdateResult.java       |  65 +++
 .../placement/ResourceRequestUpdateResult.java  |  43 --
 .../server/resourcemanager/Application.java     |   6 +-
 .../yarn/server/resourcemanager/MockNodes.java  |   4 +
 .../server/resourcemanager/RMHATestBase.java    |  30 +-
 .../server/resourcemanager/TestAppManager.java  | 110 +++-
 .../resourcemanager/TestClientRMTokens.java     |  44 ++
 .../applicationsmanager/MockAsm.java            |   7 +
 .../monitor/TestSchedulingMonitor.java          |  41 ++
 ...estProportionalCapacityPreemptionPolicy.java |  22 +-
 .../recovery/TestRMStateStoreUtils.java         |  81 +++
 .../server/resourcemanager/rmapp/MockRMApp.java |   7 +
 .../rmcontainer/TestRMContainerImpl.java        |   8 +-
 .../scheduler/TestClusterNodeTracker.java       | 125 ++++-
 .../capacity/TestCapacityScheduler.java         |   3 +-
 .../TestCapacitySchedulerLazyPreemption.java    |  36 +-
 ...TestCapacitySchedulerSurgicalPreemption.java |  40 +-
 .../distributed/TestNodeQueueLoadMonitor.java   |  21 +
 ...ebServicesDelegationTokenAuthentication.java |  15 +-
 .../src/site/markdown/CapacityScheduler.md      |  10 +
 .../site/markdown/OpportunisticContainers.md    | 212 --------
 .../site/markdown/OpportunisticContainers.md.vm | 233 ++++++++
 .../site/markdown/yarn-service/QuickStart.md    |  20 +-
 .../markdown/yarn-service/YarnServiceAPI.md     |  11 +-
 .../src/main/webapp/app/adapters/yarn-log.js    |  39 ++
 .../webapp/app/components/collapsible-panel.js  |  64 +++
 .../main/webapp/app/components/timeline-view.js |   6 +-
 .../main/webapp/app/components/tree-selector.js |   2 +-
 .../webapp/app/controllers/yarn-app-attempt.js  |   2 +-
 .../webapp/app/controllers/yarn-app/logs.js     | 204 +++++++
 .../main/webapp/app/controllers/yarn-queue.js   |   6 +-
 .../webapp/app/controllers/yarn-queue/apps.js   |   6 +-
 .../src/main/webapp/app/models/yarn-log.js      |  27 +
 .../app/models/yarn-queue/capacity-queue.js     |  11 +-
 .../src/main/webapp/app/models/yarn-rm-node.js  |   6 +-
 .../src/main/webapp/app/router.js               |   1 +
 .../main/webapp/app/routes/yarn-app-attempt.js  |   4 +-
 .../main/webapp/app/routes/yarn-app/attempts.js |   4 +-
 .../src/main/webapp/app/routes/yarn-app/logs.js |  47 ++
 .../src/main/webapp/app/serializers/yarn-log.js |  48 ++
 .../src/main/webapp/app/styles/app.scss         |  58 +-
 .../src/main/webapp/app/styles/compose-box.scss |  39 ++
 .../src/main/webapp/app/styles/layout.scss      |   4 +
 .../src/main/webapp/app/styles/variables.scss   |   3 +-
 .../templates/components/collapsible-panel.hbs  |  21 +
 .../templates/components/node-menu-panel.hbs    |   2 +-
 .../app/templates/components/timeline-view.hbs  |  66 +--
 .../yarn-queue/capacity-queue-info.hbs          |  51 +-
 .../components/yarn-queue/capacity-queue.hbs    |  81 +--
 .../components/yarn-queue/fair-queue.hbs        |  66 +--
 .../components/yarn-queue/fifo-queue.hbs        |  43 +-
 .../webapp/app/templates/yarn-app-attempt.hbs   |  21 +-
 .../src/main/webapp/app/templates/yarn-app.hbs  |   3 +
 .../webapp/app/templates/yarn-app/attempts.hbs  |  13 +
 .../main/webapp/app/templates/yarn-app/logs.hbs | 132 +++++
 .../webapp/app/templates/yarn-node-apps.hbs     |   2 +-
 .../app/templates/yarn-node-containers.hbs      |   2 +-
 .../app/templates/yarn-node/yarn-nm-gpu.hbs     |   4 +
 .../main/webapp/app/templates/yarn-queue.hbs    |  73 ++-
 .../webapp/app/templates/yarn-queue/apps.hbs    |  15 +-
 .../webapp/app/templates/yarn-queue/info.hbs    |  17 +-
 .../main/webapp/app/templates/yarn-queues.hbs   |   5 +-
 .../src/main/webapp/bower-shrinkwrap.json       |  12 +-
 .../components/collapsible-panel-test.js        |  43 ++
 .../webapp/tests/unit/adapters/yarn-log-test.js |  30 ++
 .../unit/controllers/yarn-app/logs-test.js      |  30 ++
 .../webapp/tests/unit/models/yarn-log-test.js   |  30 ++
 .../tests/unit/routes/yarn-app/logs-test.js     |  29 +
 .../tests/unit/serializers/yarn-log-test.js     |  33 ++
 298 files changed, 12277 insertions(+), 3476 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f16cdbd/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
index 8e5b6ee,8e2bc94..5503cb2
--- a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
@@@ -61,10 -63,33 +63,39 @@@
      <Bug pattern="IS2_INCONSISTENT_SYNC" />
    </Match>
  
 +  <Match>
 +    <Package name="org.apache.hadoop.ozone.protocol.proto" />
 +  </Match>
 +  <Match>
 +    <Package name="org.apache.hadoop.hdfs.ozone.protocol.proto" />
 +  </Match>
++
+   <!-- BlockLocations are user-facing, but LocatedBlocks are not. -->
+   <Match>
+     <Class name="org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus" />
+     <Field name="hdfsloc" />
+     <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED" />
+   </Match>
+ 
+   <!-- Hdfs*FileStatus are internal types. This "internal" state is not sensitive. -->
+   <Match>
+     <Class name="org.apache.hadoop.hdfs.protocol.HdfsNamedFileStatus" />
+     <Method name="getLocalNameInBytes" />
+     <Bug pattern="EI_EXPOSE_REP" />
+   </Match>
+   <Match>
+     <Class name="org.apache.hadoop.hdfs.protocol.HdfsNamedFileStatus" />
+     <Method name="getSymlinkInBytes" />
+     <Bug pattern="EI_EXPOSE_REP" />
+   </Match>
+   <Match>
+     <Class name="org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus" />
+     <Method name="getLocalNameInBytes" />
+     <Bug pattern="EI_EXPOSE_REP" />
+   </Match>
+   <Match>
+     <Class name="org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus" />
+     <Method name="getSymlinkInBytes" />
+     <Bug pattern="EI_EXPOSE_REP" />
+   </Match>
 -
  </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f16cdbd/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f16cdbd/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f16cdbd/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f16cdbd/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f16cdbd/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f16cdbd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f16cdbd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index fadc554,2d710be..b8157c8
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@@ -3108,12 -3100,21 +3108,22 @@@ public class MiniDFSCluster implements 
      // Wait for new namenode to get registrations from all the datanodes
      waitActive(nnIndex);
    }
+ 
+   /**
+    * Sets the timeout for re-issuing a block recovery.
+    */
+   public void setBlockRecoveryTimeout(long timeout) {
+     for (int nnIndex = 0; nnIndex < getNumNameNodes(); nnIndex++) {
+       getNamesystem(nnIndex).getBlockManager().setBlockRecoveryTimeout(
+           timeout);
+     }
+   }
    
 -  protected void setupDatanodeAddress(Configuration conf, boolean setupHostsFile,
 -                           boolean checkDataNodeAddrConfig) throws IOException {
 +  protected void setupDatanodeAddress(
 +      int i, Configuration dnConf, boolean setupHostsFile,
 +      boolean checkDataNodeAddrConfig) throws IOException {
      if (setupHostsFile) {
 -      String hostsFile = conf.get(DFS_HOSTS, "").trim();
 +      String hostsFile = dnConf.get(DFS_HOSTS, "").trim();
        if (hostsFile.length() == 0) {
          throw new IOException("Parameter dfs.hosts is not setup in conf");
        }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f16cdbd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f16cdbd/hadoop-project/pom.xml
----------------------------------------------------------------------


---------------------------------------------------------------------
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: YARN-7611. Node manager web UI should display container type in containers page. Contributed by Weiwei Yang.

Posted by ae...@apache.org.
YARN-7611. Node manager web UI should display container type in containers page. 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/05c347fe
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/05c347fe
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/05c347fe

Branch: refs/heads/HDFS-7240
Commit: 05c347fe51c01494ed8110f8f116a01c90205f13
Parents: 73b8697
Author: Weiwei Yang <ww...@apache.org>
Authored: Wed Dec 6 12:21:52 2017 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Wed Dec 6 12:21:52 2017 +0800

----------------------------------------------------------------------
 .../yarn/server/nodemanager/webapp/AllContainersPage.java      | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/05c347fe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllContainersPage.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/AllContainersPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllContainersPage.java
index 3fc6f3c..afc9692 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllContainersPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllContainersPage.java
@@ -51,9 +51,9 @@ public class AllContainersPage extends NMView {
 
   private String containersTableInit() {
     return tableInit().
-        // containerid, containerid, log-url
+        // containerid, executiontype, containerid, log-url
         append(", aoColumns:[").append(getContainersIdColumnDefs())
-        .append(", null, {bSearchable:false}]} ").toString();
+        .append(", null, null, {bSearchable:false}]} ").toString();
   }
 
   private String getContainersIdColumnDefs() {
@@ -83,6 +83,7 @@ public class AllContainersPage extends NMView {
           .thead()
             .tr()
               .td().__("ContainerId").__()
+              .td().__("ExecutionType").__()
               .td().__("ContainerState").__()
               .td().__("logs").__()
             .__()
@@ -94,6 +95,7 @@ public class AllContainersPage extends NMView {
           .tr()
             .td().a(url("container", info.getId()), info.getId())
             .__()
+            .td().__(info.getExecutionType()).__()
             .td().__(info.getState()).__()
             .td()
                 .a(url(info.getShortLogLink()), "logs").__()


---------------------------------------------------------------------
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: YARN-7558. yarn logs command fails to get logs for running containers if UI authentication is enabled. Contributed by Xuan Gong.

Posted by ae...@apache.org.
YARN-7558. yarn logs command fails to get logs for running containers if UI authentication is enabled. 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/a4094259
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a4094259
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a4094259

Branch: refs/heads/HDFS-7240
Commit: a409425986fc128bb54f656b05373201545f7213
Parents: b1c7654
Author: Junping Du <ju...@apache.org>
Authored: Thu Nov 30 13:47:47 2017 -0800
Committer: Junping Du <ju...@apache.org>
Committed: Thu Nov 30 13:47:47 2017 -0800

----------------------------------------------------------------------
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  | 41 ++++++++++++++------
 1 file changed, 29 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4094259/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
index 74497ce..6953a4d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
@@ -18,13 +18,25 @@
 
 package org.apache.hadoop.yarn.client.cli;
 
+import com.google.common.annotations.VisibleForTesting;
+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.UniformInterfaceException;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.filter.ClientFilter;
+import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
+import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
 import java.net.ConnectException;
+import java.net.HttpURLConnection;
 import java.net.SocketException;
 import java.net.SocketTimeoutException;
+import java.net.URL;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -38,9 +50,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.regex.Pattern;
-
 import javax.ws.rs.core.MediaType;
-
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.GnuParser;
@@ -57,6 +67,8 @@ import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -78,15 +90,6 @@ import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 
-import com.google.common.annotations.VisibleForTesting;
-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.UniformInterfaceException;
-import com.sun.jersey.api.client.WebResource;
-import com.sun.jersey.api.client.filter.ClientFilter;
-
 @Public
 @Evolving
 public class LogsCLI extends Configured implements Tool {
@@ -132,7 +135,21 @@ public class LogsCLI extends Configured implements Tool {
   public int run(String[] args) throws Exception {
     try {
       yarnClient = createYarnClient();
-      webServiceClient = Client.create();
+      webServiceClient = new Client(new URLConnectionClientHandler(
+          new HttpURLConnectionFactory() {
+          @Override
+          public HttpURLConnection getHttpURLConnection(URL url)
+              throws IOException {
+            AuthenticatedURL.Token token = new AuthenticatedURL.Token();
+            HttpURLConnection conn = null;
+            try {
+              conn = new AuthenticatedURL().openConnection(url, token);
+            } catch (AuthenticationException e) {
+              throw new IOException(e);
+            }
+            return conn;
+          }
+        }));
       return runCommand(args);
     } finally {
       if (yarnClient != null) {


---------------------------------------------------------------------
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-7546. Layout changes in Queue UI to show queue details on right pane. Contributed by Vasudevan Skm.

Posted by ae...@apache.org.
YARN-7546. Layout changes in Queue UI to show queue details on right pane. Contributed by Vasudevan Skm.


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

Branch: refs/heads/HDFS-7240
Commit: 4653aa3eb31fb23fa19136173685464d71f86613
Parents: 60fd0d7
Author: Sunil G <su...@apache.org>
Authored: Fri Dec 1 13:26:01 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Dec 1 13:26:01 2017 +0530

----------------------------------------------------------------------
 .../main/webapp/app/components/tree-selector.js |  2 +-
 .../main/webapp/app/controllers/yarn-queue.js   |  6 +-
 .../webapp/app/controllers/yarn-queue/apps.js   |  6 +-
 .../app/models/yarn-queue/capacity-queue.js     | 11 ++-
 .../src/main/webapp/app/styles/app.scss         | 58 +++++++++++++-
 .../src/main/webapp/app/styles/compose-box.scss | 39 ++++++++++
 .../src/main/webapp/app/styles/layout.scss      |  4 +
 .../src/main/webapp/app/styles/variables.scss   |  3 +-
 .../yarn-queue/capacity-queue-info.hbs          | 51 +++---------
 .../components/yarn-queue/capacity-queue.hbs    | 81 +++++++++++---------
 .../components/yarn-queue/fair-queue.hbs        | 66 ++++++++--------
 .../components/yarn-queue/fifo-queue.hbs        | 43 ++++++-----
 .../main/webapp/app/templates/yarn-queue.hbs    | 73 ++++++++++++------
 .../webapp/app/templates/yarn-queue/apps.hbs    | 15 +++-
 .../webapp/app/templates/yarn-queue/info.hbs    | 17 ++--
 .../main/webapp/app/templates/yarn-queues.hbs   |  5 +-
 16 files changed, 300 insertions(+), 180 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4653aa3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
index 7a9d53b..4645a48 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
@@ -146,7 +146,7 @@ export default Ember.Component.extend({
 
       }.bind(this))
     .on("dblclick", function (d) {
-      document.location.href = "#/yarn-queue/" + d.queueData.get("name") + "/info";
+      document.location.href = "#/yarn-queue/" + d.queueData.get("name") + "/apps";
     });
 
     nodeEnter.append("circle")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4653aa3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queue.js
index 3a72b60..e9f945b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queue.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queue.js
@@ -33,15 +33,11 @@ export default Ember.Controller.extend({
       text: "Queues",
       routeName: 'yarn-queues',
       model: 'root'
-    }, {
-      text: `Queue [ ${queueName} ]`,
-      routeName: 'yarn-queue.info',
-      model: queueName
     }];
 
     if (path && path === "yarn-queue.apps") {
       crumbs.push({
-        text: "Applications",
+        text: `Queue [ ${queueName} ]`,
         routeName: 'yarn-queue.apps',
         model: queueName
       });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4653aa3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queue/apps.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queue/apps.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queue/apps.js
index 905d96d..c10624e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queue/apps.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queue/apps.js
@@ -21,8 +21,10 @@ import TableDefinition from 'em-table/utils/table-definition';
 import AppTableController from '../app-table-columns';
 
 export default AppTableController.extend({
-  // Your custom instance of table definition
-  tableDefinition: TableDefinition.create(),
+  tableDefinition: TableDefinition.create({
+    enableFaceting: true,
+    rowCount: 25
+  }),
 
   // Search text alias, any change in controller's searchText would affect the table's searchText, and vice-versa.
   _selectedObserver: Ember.on("init", Ember.observer("model.selected", function () {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4653aa3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js
index 1d162e9..f892c2b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js
@@ -51,15 +51,18 @@ export default DS.Model.extend({
     var floatToFixed = Converter.floatToFixed;
     return [
       {
-        label: "Absolute Capacity",
-        value: this.get("name") === "root" ? 100 : floatToFixed(this.get("absCapacity"))
-      },
-      {
         label: "Absolute Used",
+        style: "primary",
         value: this.get("name") === "root" ? floatToFixed(this.get("usedCapacity")) : floatToFixed(this.get("absUsedCapacity"))
       },
       {
+        label: "Absolute Capacity",
+        style: "primary",
+        value: this.get("name") === "root" ? 100 : floatToFixed(this.get("absCapacity"))
+      },
+      {
         label: "Absolute Max Capacity",
+        style: "secondary",
         value: this.get("name") === "root" ? 100 : floatToFixed(this.get("absMaxCapacity"))
       }
     ];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4653aa3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.scss
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.scss b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.scss
index 471e346..87ee9a9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.scss
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.scss
@@ -1,6 +1,7 @@
 @import 'variables.scss';
 @import 'layout.scss';
 @import 'yarn-app.scss';
+@import './compose-box.scss';
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -191,7 +192,7 @@ table.dataTable thead .sorting_desc_disabled {
 
 .breadcrumb {
   padding-bottom: 3px;
-  background-color: none;
+  background: none;
 }
 
 .navbar-default .navbar-nav > li > a {
@@ -744,4 +745,57 @@ div.service-action-mask img {
   background: none;
   border: none;
   box-shadow: none;
-}
\ No newline at end of file
+}
+
+.queue-page-breadcrumb,
+#tree-selector-container {
+  width: calc(100% - #{$compose-box-width});
+}
+
+#tree-selector-container {
+  overflow: scroll;
+}
+
+.flex {
+  display: flex;
+}
+
+.yarn-label {
+  border-radius: 3px;
+  margin-bottom: 5px;
+  border: 1px solid $yarn-panel-bg;
+  font-size: 12px;
+  > span {
+    padding: 5px;
+  }
+  &.primary {
+    display: inline-grid;
+    .label-key {
+      color: $yarn-panel-bg;
+      background: #666;
+    }
+    .label-value {
+      color: $yarn-panel-bg;
+      background: $yarn-success-border;
+    }
+  }
+  &.secondary {
+    display: inline-table;
+    .label-key {
+      color: $yarn-panel-bg;
+      background: #999;
+    }
+
+    .label-value {
+      color: $yarn-panel-bg;
+      background: yellowgreen;
+    }
+  }
+}
+
+.yarn-queues-container {
+  padding: 15px;
+  h3 {
+    margin-top: 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4653aa3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/compose-box.scss
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/compose-box.scss b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/compose-box.scss
new file mode 100644
index 0000000..0bfadb0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/compose-box.scss
@@ -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 'variables.scss';
+
+.yarn-compose-box {
+  position: fixed;
+  bottom: 0;
+  top: 0px;
+  right: 0px;
+  background-color: $yarn-panel-bg;
+  border: 1px solid $yarn-border-color;
+  border-radius: 3px;
+  box-shadow: 0 1px 1px rgba(0, 0, 0, 0.05);
+  max-width: $compose-box-width;
+  overflow: scroll;
+
+  .panel-heading {
+    background-color: $yarn-header-bg !important;
+    border-color: $yarn-border-color;
+    border-radius: 3px;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4653aa3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/layout.scss
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/layout.scss b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/layout.scss
index d31f145..587df66 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/layout.scss
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/layout.scss
@@ -40,3 +40,7 @@
 .tail-2 {
   margin-right: 10px
 }
+
+.top-1 {
+  margin-top: 5px;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4653aa3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/variables.scss
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/variables.scss b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/variables.scss
index 8226770..e25b482 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/variables.scss
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/variables.scss
@@ -37,8 +37,7 @@ $yarn-warn-border: $color-yellow-secondary;
 $yarn-warn-bg: $color-yellow-primary;
 
 $yarn-gray-icon: $color-gray-40;
-
 $yarn-muted-text: $color-gray-60;
 
-
+$compose-box-width: 400px;
 //shadows

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4653aa3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-info.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-info.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-info.hbs
index 7d44e69..a7260bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-info.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-info.hbs
@@ -16,60 +16,29 @@
  * limitations under the License.
 }}
 
-<div class="row">
-
-  <div class="col-lg-6 container-fluid">
+<div>
+  <div class="col-lg-6">
     <div class="panel panel-default">
       <div class="panel-heading">
-        Queue Capacities: {{model.selected}}
+        Running Apps: {{model.selected}}
       </div>
-      <div class="container-fluid" id="capacity-bar-chart">
-        <br/>
-        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
-        title=""
-        parentId="capacity-bar-chart"
-        textWidth=170
-        ratio=0.55
+      <div id="numapplications-donut-chart">
+        {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
+        showLabels=true
+        parentId="numapplications-donut-chart"
+        ratio=0.6
         maxHeight=350}}
       </div>
     </div>
   </div>
 
-  <div class="col-lg-6 container-fluid">
-    <div class="panel panel-default">
-      <div class="panel-heading">
-        Queue Information: {{model.selected}}
-      </div>
-        {{yarn-queue.capacity-queue-conf-table queue=model.selectedQueue}}
-    </div>
-  </div>
-
-</div>
-
-<div class="row">
-
-    <div class="col-lg-6 container-fluid">
-      <div class="panel panel-default">
-        <div class="panel-heading">
-          Running Apps: {{model.selected}}
-        </div>
-        <div class="container-fluid" id="numapplications-donut-chart">
-          {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
-          showLabels=true
-          parentId="numapplications-donut-chart"
-          ratio=0.6
-          maxHeight=350}}
-        </div>
-      </div>
-    </div>
-
   {{#if model.selectedQueue.hasUserUsages}}
-    <div class="col-lg-6 container-fluid">
+    <div class="col-lg-6">
       <div class="panel panel-default">
         <div class="panel-heading">
           User Usages: {{model.selected}}
         </div>
-        <div class="container-fluid"  id="userusage-donut-chart">
+        <div id="userusage-donut-chart">
           {{donut-chart data=model.selectedQueue.userUsagesDonutChartData
           showLabels=true
           parentId="userusage-donut-chart"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4653aa3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs
index 8b63b66..bb9a87e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs
@@ -19,45 +19,56 @@
 {{queue-navigator model=model.queues selected=model.selected
   used="usedCapacity" max="absMaxCapacity"}}
 
-<div class="row">
-  <div class="col-lg-4 container-fluid">
-    <div class="panel panel-default">
-      <div class="panel-heading">
-        Queue Information: {{model.selected}}
+<div class="yarn-compose-box yarn-queues-container">
+  <div>
+    <h3>
+      <a href="#/yarn-queue/{{model.selected}}/apps">
+        {{model.selected}}
+      </a>
+    </h3>
+    {{#if model.selectedQueue.state}}
+      <div>
+        {{em-table-simple-status-cell content=model.selectedQueue.state}}
       </div>
-        {{yarn-queue.capacity-queue-conf-table queue=model.selectedQueue}}
+    {{/if}}
+    <div class="top-1">
+      {{#each model.selectedQueue.capacitiesBarChartData as |item|}}
+        <span class="yarn-label {{item.style}}">
+          <span class="label-key"> {{lower item.label}}</span>
+          <span class="label-value">{{item.value}}%</span>
+        </span>
+      {{/each}}
     </div>
-  </div>
-
-  <div class="col-lg-4 container-fluid">
-    <div class="panel panel-default">
-      <div class="panel-heading">
-        Queue Capacities: {{model.selected}}
-      </div>
-      <div class="container-fluid" id="capacity-bar-chart">
-        <br/>
-        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
-        title=""
-        parentId="capacity-bar-chart"
-        textWidth=175
-        ratio=0.55
-        maxHeight=350}}
-      </div>
+    <div class="top-1">
+      <span class="yarn-label secondary">
+        <span class="label-key">configured capacity</span>
+        <span class="label-value">{{model.selectedQueue.capacity}}%</span>
+      </span>
+      <span class="yarn-label secondary">
+        <span class="label-key">configured max capacity</span>
+        <span class="label-value">{{model.selectedQueue.maxCapacity}}%</span>
+      </span>
     </div>
+    {{#if model.selectedQueue.isLeafQueue}}
+      <div class="top-1">
+        <span class="yarn-label secondary">
+          <span class="label-key">user limit</span>
+          <span class="label-value">{{model.selectedQueue.userLimit}}%</span>
+        </span>
+        <span class="yarn-label secondary">
+          <span class="label-key">user limit factor</span>
+          <span class="label-value">{{model.selectedQueue.userLimitFactor}}</span>
+        </span>
+      </div>
+    {{/if}}
   </div>
 
-  <div class="col-lg-4 container-fluid">
-    <div class="panel panel-default">
-      <div class="panel-heading">
-        Running Apps: {{model.selected}}
-      </div>
-      <div class="container-fluid" id="numapplications-donut-chart">
-        {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
-        showLabels=true
-        parentId="numapplications-donut-chart"
-        ratio=0.6
-        maxHeight=350}}
-      </div>
-    </div>
+  <h5> Running Apps </h5>
+  <div id="numapplications-donut-chart">
+    {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
+    showLabels=true
+    parentId="numapplications-donut-chart"
+    ratio=0.6
+    maxHeight=350}}
   </div>
 </div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4653aa3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue.hbs
index 6d0e994..dcc80c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue.hbs
@@ -19,44 +19,46 @@
 {{queue-navigator model=model.queues selected=model.selected
   used="usedResources.memory" max="clusterResources.memory"}}
 
-<div class="row">
-  <div class="col-lg-4 container-fluid">
-    <div class="panel panel-default">
-      <div class="panel-heading">
-        Queue Information: {{model.selected}}
+<div class="yarn-compose-box">
+  <div class="panel-heading">
+    Queue Information: {{model.selected}}
+  </div>
+  <div class="panel-body">
+    <div class="container-fluid">
+      <div class="panel panel-default">
+          {{yarn-queue.fair-queue-conf-table queue=model.selectedQueue}}
       </div>
-        {{yarn-queue.fair-queue-conf-table queue=model.selectedQueue}}
     </div>
-  </div>
 
-  <div class="col-lg-4 container-fluid">
-    <div class="panel panel-default">
-      <div class="panel-heading">
-        Queue Capacities: {{model.selected}}
-      </div>
-      <div class="container-fluid" id="capacity-bar-chart">
-        <br/>
-        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
-        title=""
-        parentId="capacity-bar-chart"
-        textWidth=175
-        ratio=0.55
-        maxHeight=350}}
+    <div class="container-fluid">
+      <div class="panel panel-default">
+        <div class="panel-heading">
+          Queue Capacities: {{model.selected}}
+        </div>
+        <div class="container-fluid" id="capacity-bar-chart">
+          <br/>
+          {{bar-chart data=model.selectedQueue.capacitiesBarChartData
+          title=""
+          parentId="capacity-bar-chart"
+          textWidth=175
+          ratio=0.55
+          maxHeight=350}}
+        </div>
       </div>
     </div>
-  </div>
 
-  <div class="col-lg-4 container-fluid">
-    <div class="panel panel-default">
-      <div class="panel-heading">
-        Running Apps: {{model.selected}}
-      </div>
-      <div class="container-fluid" id="numapplications-donut-chart">
-        {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
-        showLabels=true
-        parentId="numapplications-donut-chart"
-        ratio=0.6
-        maxHeight=350}}
+    <div class="container-fluid">
+      <div class="panel panel-default">
+        <div class="panel-heading">
+          Running Apps: {{model.selected}}
+        </div>
+        <div class="container-fluid" id="numapplications-donut-chart">
+          {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
+          showLabels=true
+          parentId="numapplications-donut-chart"
+          ratio=0.6
+          maxHeight=350}}
+        </div>
       </div>
     </div>
   </div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4653aa3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue.hbs
index 90cbd27..98db5cb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue.hbs
@@ -19,29 +19,30 @@
 {{queue-navigator model=model.queues selected=model.selected
   used="usedNodeCapacity" max="totalNodeCapacity"}}
 
-<div class="row">
-  <div class="col-lg-6 container-fluid">
-    <div class="panel panel-default">
-      <div class="panel-heading">
-        Queue Information: {{model.selected}}
-      </div>
-        {{yarn-queue.fifo-queue-conf-table queue=model.selectedQueue}}
-    </div>
+<div class="yarn-compose-box">
+  <div class="panel-heading">
+    Queue Information: {{model.selected}}
   </div>
-
-  <div class="col-lg-6 container-fluid">
-    <div class="panel panel-default">
-      <div class="panel-heading">
-        Queue Capacities: {{model.selected}}
+  <div class="panel-body">
+    <div class="container-fluid">
+      <div class="panel panel-default">
+          {{yarn-queue.fifo-queue-conf-table queue=model.selectedQueue}}
       </div>
-      <div class="container-fluid" id="capacity-bar-chart">
-        <br/>
-        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
-        title=""
-        parentId="capacity-bar-chart"
-        textWidth=175
-        ratio=0.55
-        maxHeight=350}}
+    </div>
+    <div class="container-fluid">
+      <div class="panel panel-default">
+        <div class="panel-heading">
+          Queue Capacities: {{model.selected}}
+        </div>
+        <div class="container-fluid" id="capacity-bar-chart">
+          <br/>
+          {{bar-chart data=model.selectedQueue.capacitiesBarChartData
+          title=""
+          parentId="capacity-bar-chart"
+          textWidth=175
+          ratio=0.55
+          maxHeight=350}}
+        </div>
       </div>
     </div>
   </div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4653aa3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue.hbs
index ef2d285..87b596e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue.hbs
@@ -18,34 +18,61 @@
 
 {{breadcrumb-bar breadcrumbs=breadcrumbs}}
 
-<div class="col-md-12 container-fluid">
-  <div class="row">
-
-    <div class="col-md-2 container-fluid">
-      <div class="panel panel-default">
-        <div class="panel-heading">
-          <h4>Queue</h4>
+<div class="panel-group">
+  <div class="panel panel-default">
+    <div class="yarn-app-header">
+      <div class="flex">
+        <div class="top-1">
+          <h3>{{model.selected}}</h3>
+          {{#if model.selectedQueue.state}}
+          <div>
+            {{em-table-simple-status-cell content=model.selectedQueue.state}}
+          </div>
+          {{/if}}
+          <div class="top-1">
+            <span class="yarn-label secondary">
+              <span class="label-key">configured capacity</span>
+              <span class="label-value">{{model.selectedQueue.capacity}}%</span>
+            </span>
+            <span class="yarn-label secondary">
+              <span class="label-key">configured max capacity</span>
+              <span class="label-value">{{model.selectedQueue.maxCapacity}}%</span>
+            </span>
+            {{#if model.selectedQueue.isLeafQueue}}
+            <span class="yarn-label secondary">
+              <span class="label-key">user limit</span>
+              <span class="label-value">{{model.selectedQueue.userLimit}}%</span>
+            </span>
+            <span class="yarn-label secondary">
+              <span class="label-key">user limit factor</span>
+              <span class="label-value">{{model.selectedQueue.userLimitFactor}}</span>
+            </span>
+            {{/if}}
+          </div>
         </div>
-        <div class="panel-body">
-          <ul class="nav nav-pills nav-stacked" id="stacked-menu">
-            <ul class="nav nav-pills nav-stacked collapse in">
-              {{#link-to 'yarn-queue.info' tagName="li"}}
-                {{#link-to 'yarn-queue.info' model.selected}}Information
-                {{/link-to}}
-              {{/link-to}}
-              {{#link-to 'yarn-queue.apps' tagName="li"}}
-                {{#link-to 'yarn-queue.apps' model.selected}}Applications List
-                {{/link-to}}
-              {{/link-to}}
-            </ul>
-          </ul>
+        <div class="flex-right">
+          {{#each model.selectedQueue.capacitiesBarChartData as |item|}}
+            <span class="yarn-label primary">
+              <span class="label-key"> {{lower item.label}}</span>
+              <span class="label-value">{{item.value}}%</span>
+            </span>
+          {{/each}}
         </div>
       </div>
     </div>
-
-    <div class="col-md-10 container-fluid">
+        <div class="panel-heading">
+      <div class="clearfix">
+        <ul class="nav nav-pills">
+          <ul class="nav nav-pills collapse in">
+            {{#link-to 'yarn-queue.apps' tagName="li" class=(if (eq target.currentPath 'yarn-queue.apps') "active")}}
+              {{#link-to 'yarn-queue.apps' appId (query-params service=model.serviceName)}}Applications{{/link-to}}
+            {{/link-to}}
+          </ul>
+        </ul>
+      </div>
+    </div>
+    <div class="panel-body yarn-app-body">
       {{outlet}}
     </div>
-
   </div>
 </div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4653aa3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/apps.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/apps.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/apps.hbs
index 4a508c1..6417910 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/apps.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/apps.hbs
@@ -17,9 +17,20 @@
 }}
 
 <div class="row">
-  <div class="col-lg-12 container-fluid">
+  <div class="col-lg-12">
+    <div class="row">
+      {{#if (eq model.queues.firstObject.type "capacity")}}
+        {{yarn-queue.capacity-queue-info model=model}}
+      {{else if (eq model.queues.firstObject.type "fair")}}
+        {{yarn-queue.fair-queue-info model=model}}
+      {{else}}
+        {{yarn-queue.fifo-queue-info model=model}}
+      {{/if}}
+    </div>
+  </div>
+  <div class="col-lg-12 yarn-applications-container">
     {{#if model.apps}}
-      {{em-table columns=columns rows=model.apps}}
+      {{em-table columns=columns rows=model.apps  definition=tableDefinitio}}
     {{else}}
       <h4 align = "center">Could not find any applications from this cluster</h4>
     {{/if}}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4653aa3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/info.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/info.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/info.hbs
index 2f138a7..b84425a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/info.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/info.hbs
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
 }}
-
-{{#if (eq model.queues.firstObject.type "capacity")}}
-  {{yarn-queue.capacity-queue-info model=model}}
-{{else if (eq model.queues.firstObject.type "fair")}}
-  {{yarn-queue.fair-queue-info model=model}}
-{{else}}
-  {{yarn-queue.fifo-queue-info model=model}}
-{{/if}}
+<div class="row">
+  {{#if (eq model.queues.firstObject.type "capacity")}}
+    {{yarn-queue.capacity-queue-info model=model}}
+  {{else if (eq model.queues.firstObject.type "fair")}}
+    {{yarn-queue.fair-queue-info model=model}}
+  {{else}}
+    {{yarn-queue.fifo-queue-info model=model}}
+  {{/if}}
+</div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4653aa3e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs
index fccdb5b..b3165d5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs
@@ -15,8 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
 }}
-
-{{breadcrumb-bar breadcrumbs=breadcrumbs}}
+<div class="queue-page-breadcrumb">
+  {{breadcrumb-bar breadcrumbs=breadcrumbs}}
+</div>
 <div class="container-fluid">
   {{#if (eq model.queues.firstObject.type "capacity")}}
     {{yarn-queue.capacity-queue model=model}}


---------------------------------------------------------------------
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: YARN-5594. Handle old RMDelegationToken format when recovering RM (rkanter)

Posted by ae...@apache.org.
YARN-5594. Handle old RMDelegationToken format when recovering RM (rkanter)


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

Branch: refs/heads/HDFS-7240
Commit: d8863fc16fa3cbcdda5b99f79386c43e4fae5917
Parents: d30d578
Author: Robert Kanter <rk...@apache.org>
Authored: Mon Dec 4 13:14:55 2017 -0800
Committer: Robert Kanter <rk...@apache.org>
Committed: Mon Dec 4 13:14:55 2017 -0800

----------------------------------------------------------------------
 .../client/YARNDelegationTokenIdentifier.java   |  11 ++
 .../yarn/security/TestYARNTokenIdentifier.java  | 115 +++++++++++--------
 .../recovery/FileSystemRMStateStore.java        |   4 +-
 .../recovery/LeveldbRMStateStore.java           |   5 +-
 .../recovery/RMStateStoreUtils.java             |  69 +++++++++++
 .../recovery/ZKRMStateStore.java                |   3 +-
 .../RMDelegationTokenIdentifierData.java        |   8 ++
 .../resourcemanager/TestClientRMTokens.java     |  44 +++++++
 .../recovery/TestRMStateStoreUtils.java         |  81 +++++++++++++
 9 files changed, 282 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8863fc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/YARNDelegationTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/YARNDelegationTokenIdentifier.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/YARNDelegationTokenIdentifier.java
index 40ea858..da6a8c5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/YARNDelegationTokenIdentifier.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/YARNDelegationTokenIdentifier.java
@@ -22,6 +22,7 @@ import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
@@ -64,6 +65,11 @@ public abstract class YARNDelegationTokenIdentifier extends
     setMasterKeyId(builder.getMasterKeyId());
   }
 
+  public synchronized void readFieldsInOldFormat(DataInput in)
+      throws IOException {
+    super.readFields(in);
+  }
+
   private void setBuilderFields() {
     if (builder.getOwner() != null &&
         !builder.getOwner().equals(getOwner().toString())) {
@@ -97,6 +103,11 @@ public abstract class YARNDelegationTokenIdentifier extends
     builder.build().writeTo((DataOutputStream) out);
   }
 
+  @VisibleForTesting
+  public synchronized void writeInOldFormat(DataOutput out) throws IOException {
+    super.write(out);
+  }
+
   public YARNDelegationTokenIdentifierProto getProto() {
     setBuilderFields();
     return builder.build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8863fc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java
index 130a65e..82e1943 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java
@@ -24,6 +24,7 @@ import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.HadoopKerberosName;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -214,10 +215,20 @@ public class TestYARNTokenIdentifier {
     Assert.assertEquals(ExecutionType.GUARANTEED,
         anotherToken.getExecutionType());
   }
-  
+
   @Test
   public void testRMDelegationTokenIdentifier() throws IOException {
-    
+    testRMDelegationTokenIdentifier(false);
+  }
+
+  @Test
+  public void testRMDelegationTokenIdentifierOldFormat() throws IOException {
+    testRMDelegationTokenIdentifier(true);
+  }
+
+  public void testRMDelegationTokenIdentifier(boolean oldFormat)
+      throws IOException {
+
     Text owner = new Text("user1");
     Text renewer = new Text("user2");
     Text realUser = new Text("user3");
@@ -225,59 +236,63 @@ public class TestYARNTokenIdentifier {
     long maxDate = 2;
     int sequenceNumber = 3;
     int masterKeyId = 4;
-    
-    RMDelegationTokenIdentifier token = 
+
+    RMDelegationTokenIdentifier originalToken =
         new RMDelegationTokenIdentifier(owner, renewer, realUser);
-    token.setIssueDate(issueDate);
-    token.setMaxDate(maxDate);
-    token.setSequenceNumber(sequenceNumber);
-    token.setMasterKeyId(masterKeyId);
-    
-    RMDelegationTokenIdentifier anotherToken = new RMDelegationTokenIdentifier();
-    
-    byte[] tokenContent = token.getBytes();
-    DataInputBuffer dib = new DataInputBuffer();
-    dib.reset(tokenContent, tokenContent.length);
-    anotherToken.readFields(dib);
-    dib.close();
+    originalToken.setIssueDate(issueDate);
+    originalToken.setMaxDate(maxDate);
+    originalToken.setSequenceNumber(sequenceNumber);
+    originalToken.setMasterKeyId(masterKeyId);
+
+    RMDelegationTokenIdentifier anotherToken
+        = new RMDelegationTokenIdentifier();
+
+    if (oldFormat) {
+      DataInputBuffer inBuf = new DataInputBuffer();
+      DataOutputBuffer outBuf = new DataOutputBuffer();
+      originalToken.writeInOldFormat(outBuf);
+      inBuf.reset(outBuf.getData(), 0, outBuf.getLength());
+      anotherToken.readFieldsInOldFormat(inBuf);
+      inBuf.close();
+    } else {
+      byte[] tokenContent = originalToken.getBytes();
+      DataInputBuffer dib = new DataInputBuffer();
+      dib.reset(tokenContent, tokenContent.length);
+      anotherToken.readFields(dib);
+      dib.close();
+    }
     // verify the whole record equals with original record
-    Assert.assertEquals("Token is not the same after serialization " +
-        "and deserialization.", token, anotherToken);
-    
-    Assert.assertEquals("owner from proto is not the same with original token",
-        anotherToken.getOwner(), owner);
-    
-    Assert.assertEquals("renewer from proto is not the same with original token",
-        anotherToken.getRenewer(), renewer);
-    
-    Assert.assertEquals("realUser from proto is not the same with original token",
-        anotherToken.getRealUser(), realUser);
-    
-    Assert.assertEquals("issueDate from proto is not the same with original token",
-        anotherToken.getIssueDate(), issueDate);
-    
-    Assert.assertEquals("maxDate from proto is not the same with original token",
-        anotherToken.getMaxDate(), maxDate);
-    
-    Assert.assertEquals("sequenceNumber from proto is not the same with original token",
-        anotherToken.getSequenceNumber(), sequenceNumber);
-    
-    Assert.assertEquals("masterKeyId from proto is not the same with original token",
-        anotherToken.getMasterKeyId(), masterKeyId);
-    
-    // Test getProto    
-    RMDelegationTokenIdentifier token1 = 
-        new RMDelegationTokenIdentifier(owner, renewer, realUser);
-    token1.setIssueDate(issueDate);
-    token1.setMaxDate(maxDate);
-    token1.setSequenceNumber(sequenceNumber);
-    token1.setMasterKeyId(masterKeyId);
-    YARNDelegationTokenIdentifierProto tokenProto = token1.getProto();
+    Assert.assertEquals(
+        "Token is not the same after serialization and deserialization.",
+        originalToken, anotherToken);
+    Assert.assertEquals(
+        "owner from proto is not the same with original token",
+        owner, anotherToken.getOwner());
+    Assert.assertEquals(
+        "renewer from proto is not the same with original token",
+        renewer, anotherToken.getRenewer());
+    Assert.assertEquals(
+        "realUser from proto is not the same with original token",
+        realUser, anotherToken.getRealUser());
+    Assert.assertEquals(
+        "issueDate from proto is not the same with original token",
+        issueDate, anotherToken.getIssueDate());
+    Assert.assertEquals(
+        "maxDate from proto is not the same with original token",
+        maxDate, anotherToken.getMaxDate());
+    Assert.assertEquals(
+        "sequenceNumber from proto is not the same with original token",
+        sequenceNumber, anotherToken.getSequenceNumber());
+    Assert.assertEquals(
+        "masterKeyId from proto is not the same with original token",
+        masterKeyId, anotherToken.getMasterKeyId());
+
+    // Test getProto
+    YARNDelegationTokenIdentifierProto tokenProto = originalToken.getProto();
     // Write token proto to stream
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream out = new DataOutputStream(baos);
     tokenProto.writeTo(out);
-
     // Read token
     byte[] tokenData = baos.toByteArray();
     RMDelegationTokenIdentifier readToken = new RMDelegationTokenIdentifier();
@@ -287,7 +302,7 @@ public class TestYARNTokenIdentifier {
 
     // Verify if read token equals with original token
     Assert.assertEquals("Token from getProto is not the same after " +
-        "serialization and deserialization.", token1, readToken);
+        "serialization and deserialization.", originalToken, readToken);
     db.close();
     out.close();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8863fc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
index 7cbeda3..19297bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
@@ -378,12 +378,10 @@ public class FileSystemRMStateStore extends RMStateStore {
           }
         } else if (childNodeName.startsWith(DELEGATION_TOKEN_PREFIX)) {
           RMDelegationTokenIdentifierData identifierData =
-              new RMDelegationTokenIdentifierData();
-          identifierData.readFields(fsIn);
+              RMStateStoreUtils.readRMDelegationTokenIdentifierData(fsIn);
           RMDelegationTokenIdentifier identifier =
               identifierData.getTokenIdentifier();
           long renewDate = identifierData.getRenewDate();
-
           rmState.rmSecretManagerState.delegationTokenState.put(identifier,
             renewDate);
           if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8863fc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
index 16ae1d3..a53083f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
@@ -410,11 +410,10 @@ public class LeveldbRMStateStore extends RMStateStore {
 
   private RMDelegationTokenIdentifierData loadDelegationToken(byte[] data)
       throws IOException {
-    RMDelegationTokenIdentifierData tokenData =
-        new RMDelegationTokenIdentifierData();
+    RMDelegationTokenIdentifierData tokenData = null;
     DataInputStream in = new DataInputStream(new ByteArrayInputStream(data));
     try {
-      tokenData.readFields(in);
+      tokenData = RMStateStoreUtils.readRMDelegationTokenIdentifierData(in);
     } finally {
       IOUtils.cleanup(LOG, in);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8863fc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreUtils.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/RMStateStoreUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreUtils.java
new file mode 100644
index 0000000..9aef794
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreUtils.java
@@ -0,0 +1,69 @@
+/**
+* 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.recovery;
+
+import com.google.protobuf.InvalidProtocolBufferException;
+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.security.client.RMDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.RMDelegationTokenIdentifierData;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+
+/**
+ * Utility methods for {@link RMStateStore} and subclasses.
+ */
+@Private
+@Unstable
+public class RMStateStoreUtils {
+
+  public static final Log LOG = LogFactory.getLog(RMStateStoreUtils.class);
+
+  /**
+   * Returns the RM Delegation Token data from the {@link DataInputStream} as a
+   * {@link RMDelegationTokenIdentifierData}.  It can handle both the current
+   * and old (non-protobuf) formats.
+   *
+   * @param fsIn The {@link DataInputStream} containing RM Delegation Token data
+   * @return An {@link RMDelegationTokenIdentifierData} containing the read in
+   * RM Delegation Token
+   */
+  public static RMDelegationTokenIdentifierData
+      readRMDelegationTokenIdentifierData(DataInputStream fsIn)
+      throws IOException {
+    RMDelegationTokenIdentifierData identifierData =
+        new RMDelegationTokenIdentifierData();
+    try {
+      identifierData.readFields(fsIn);
+    } catch (InvalidProtocolBufferException e) {
+      LOG.warn("Recovering old formatted token");
+      fsIn.reset();
+      YARNDelegationTokenIdentifier identifier =
+          new RMDelegationTokenIdentifier();
+      identifier.readFieldsInOldFormat(fsIn);
+      identifierData.setIdentifier(identifier);
+      identifierData.setRenewDate(fsIn.readLong());
+    }
+    return identifierData;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8863fc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
index 36b55e5..9073910 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
@@ -668,8 +668,7 @@ public class ZKRMStateStore extends RMStateStore {
       ByteArrayInputStream is = new ByteArrayInputStream(data);
       try (DataInputStream fsIn = new DataInputStream(is)) {
         RMDelegationTokenIdentifierData identifierData =
-            new RMDelegationTokenIdentifierData();
-        identifierData.readFields(fsIn);
+            RMStateStoreUtils.readRMDelegationTokenIdentifierData(fsIn);
         RMDelegationTokenIdentifier identifier =
             identifierData.getTokenIdentifier();
         long renewDate = identifierData.getRenewDate();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8863fc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/RMDelegationTokenIdentifierData.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/records/RMDelegationTokenIdentifierData.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/RMDelegationTokenIdentifierData.java
index 97b5c1c..8ba1df7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/RMDelegationTokenIdentifierData.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/RMDelegationTokenIdentifierData.java
@@ -58,4 +58,12 @@ public class RMDelegationTokenIdentifierData {
   public long getRenewDate() {
     return builder.getRenewDate();
   }
+
+  public void setIdentifier(YARNDelegationTokenIdentifier identifier) {
+    builder.setTokenIdentifier(identifier.getProto());
+  }
+
+  public void setRenewDate(long renewDate) {
+    builder.setRenewDate(renewDate);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8863fc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
index 06c1c42..0f085d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
@@ -36,7 +36,11 @@ import java.net.InetSocketAddress;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedExceptionAction;
 
+import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.delegation.TestDelegationToken;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.RMDelegationTokenIdentifierData;
 import org.junit.AfterClass;
 import org.junit.Assert;
 
@@ -352,6 +356,46 @@ public class TestClientRMTokens {
         false);
   }
 
+  @Test
+  public void testReadOldFormatFields() throws IOException {
+    RMDelegationTokenIdentifier token = new RMDelegationTokenIdentifier(
+        new Text("alice"), new Text("bob"), new Text("colin"));
+    token.setIssueDate(123);
+    token.setMasterKeyId(321);
+    token.setMaxDate(314);
+    token.setSequenceNumber(12345);
+    DataInputBuffer inBuf = new DataInputBuffer();
+    DataOutputBuffer outBuf = new DataOutputBuffer();
+    token.writeInOldFormat(outBuf);
+    outBuf.writeLong(42);   // renewDate
+    inBuf.reset(outBuf.getData(), 0, outBuf.getLength());
+
+    RMDelegationTokenIdentifier identifier = null;
+
+    try {
+      RMDelegationTokenIdentifierData identifierData =
+          new RMDelegationTokenIdentifierData();
+      identifierData.readFields(inBuf);
+      fail("Should have thrown a "
+          + InvalidProtocolBufferException.class.getName()
+          + " because the token is not a protobuf");
+    } catch (InvalidProtocolBufferException e) {
+      identifier = new RMDelegationTokenIdentifier();
+      inBuf.reset();
+      identifier.readFieldsInOldFormat(inBuf);
+      assertEquals(42, inBuf.readLong());
+    }
+
+    assertEquals("alice", identifier.getUser().getUserName());
+    assertEquals(new Text("bob"), identifier.getRenewer());
+    assertEquals("colin", identifier.getUser().getRealUser().getUserName());
+    assertEquals(123, identifier.getIssueDate());
+    assertEquals(321, identifier.getMasterKeyId());
+    assertEquals(314, identifier.getMaxDate());
+    assertEquals(12345, identifier.getSequenceNumber());
+
+  }
+
   @SuppressWarnings("unchecked")
   private void checkShortCircuitRenewCancel(InetSocketAddress rmAddr,
                                             InetSocketAddress serviceAddr,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8863fc1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestRMStateStoreUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestRMStateStoreUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestRMStateStoreUtils.java
new file mode 100644
index 0000000..889f74f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestRMStateStoreUtils.java
@@ -0,0 +1,81 @@
+/**
+ * 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.recovery;
+
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.RMDelegationTokenIdentifierData;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestRMStateStoreUtils {
+
+  @Test
+  public void testReadRMDelegationTokenIdentifierData()
+      throws Exception {
+    testReadRMDelegationTokenIdentifierData(false);
+  }
+
+  @Test
+  public void testReadRMDelegationTokenIdentifierDataOldFormat()
+      throws Exception {
+    testReadRMDelegationTokenIdentifierData(true);
+  }
+
+  public void testReadRMDelegationTokenIdentifierData(boolean oldFormat)
+      throws Exception {
+    RMDelegationTokenIdentifier token = new RMDelegationTokenIdentifier(
+        new Text("alice"), new Text("bob"), new Text("colin"));
+    token.setIssueDate(123);
+    token.setMasterKeyId(321);
+    token.setMaxDate(314);
+    token.setSequenceNumber(12345);
+    DataInputBuffer inBuf = new DataInputBuffer();
+    if (oldFormat) {
+      DataOutputBuffer outBuf = new DataOutputBuffer();
+      token.writeInOldFormat(outBuf);
+      outBuf.writeLong(42);   // renewDate
+      inBuf.reset(outBuf.getData(), 0, outBuf.getLength());
+    } else {
+      RMDelegationTokenIdentifierData tokenIdentifierData
+          = new RMDelegationTokenIdentifierData(token, 42);
+      byte[] data = tokenIdentifierData.toByteArray();
+      inBuf.reset(data, 0, data.length);
+    }
+
+    RMDelegationTokenIdentifierData identifierData
+        = RMStateStoreUtils.readRMDelegationTokenIdentifierData(inBuf);
+    assertEquals("Found unexpected data still in the InputStream",
+        -1, inBuf.read());
+
+    RMDelegationTokenIdentifier identifier
+        = identifierData.getTokenIdentifier();
+    assertEquals("alice", identifier.getUser().getUserName());
+    assertEquals(new Text("bob"), identifier.getRenewer());
+    assertEquals("colin", identifier.getUser().getRealUser().getUserName());
+    assertEquals(123, identifier.getIssueDate());
+    assertEquals(321, identifier.getMasterKeyId());
+    assertEquals(314, identifier.getMaxDate());
+    assertEquals(12345, identifier.getSequenceNumber());
+    assertEquals(42, identifierData.getRenewDate());
+  }
+}


---------------------------------------------------------------------
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-15058. create-release site build outputs dummy shaded jars due to skipShade. Contributed by Andrew Wang.

Posted by ae...@apache.org.
HADOOP-15058. create-release site build outputs dummy shaded jars due to skipShade. Contributed by Andrew Wang.


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

Branch: refs/heads/HDFS-7240
Commit: 5533648241b11f547ac10b64a126f3ae7a2f54c4
Parents: d8863fc
Author: Sean Mackrory <ma...@apache.org>
Authored: Mon Dec 4 15:39:33 2017 -0700
Committer: Sean Mackrory <ma...@apache.org>
Committed: Mon Dec 4 15:39:43 2017 -0700

----------------------------------------------------------------------
 dev-support/bin/create-release | 71 +++++++++++++++++++++++++------------
 1 file changed, 49 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/55336482/dev-support/bin/create-release
----------------------------------------------------------------------
diff --git a/dev-support/bin/create-release b/dev-support/bin/create-release
index 694820b..949a46f 100755
--- a/dev-support/bin/create-release
+++ b/dev-support/bin/create-release
@@ -202,6 +202,8 @@ function set_defaults
     | head -1 \
     | sed  -e 's|^ *<version>||' -e 's|</version>.*$||')
 
+  DEPLOY=false
+
   DOCKER=false
   DOCKERCACHE=false
   DOCKERFILE="${BASEDIR}/dev-support/docker/Dockerfile"
@@ -283,6 +285,7 @@ function usage
 {
   echo "--artifactsdir=[path]   Path to use to store release bits"
   echo "--asfrelease            Make an ASF release"
+  echo "--deploy                Deploy Maven artifacts using ~/.m2/settings.xml"
   echo "--docker                Use Hadoop's Dockerfile for guaranteed environment"
   echo "--dockercache           Use a Docker-private maven cache"
   echo "--logdir=[path]         Path to store logs"
@@ -304,10 +307,14 @@ function option_parse
         ASFRELEASE=true
         NATIVE=true
         SIGN=true
+        DEPLOY=true
       ;;
       --artifactsdir=*)
         ARTIFACTS_DIR=${i#*=}
       ;;
+      --deploy)
+        DEPLOY=true
+      ;;
       --docker)
         DOCKER=true
       ;;
@@ -360,6 +367,11 @@ function option_parse
     fi
   fi
 
+  if [[ "${DEPLOY}" = true && ! -f "${HOME}/.m2/settings.xml" ]]; then
+    hadoop_error "ERROR: No ~/.m2/settings.xml file, cannot deploy Maven artifacts."
+    exit 1
+  fi
+
   DOCKERCMD=$(command -v docker)
   if [[ "${DOCKER}" = true && -z "${DOCKERCMD}" ]]; then
       hadoop_error "ERROR: docker binary not found. Disabling docker mode."
@@ -452,6 +464,11 @@ function dockermode
     extrad+=("-v" "${lines}:${lines}")
   fi
 
+  if [[ "${DEPLOY}" = true ]]; then
+    modp+=("--deploy")
+    extrad+=("-v" "${HOME}/.m2/settings.xml:/home/${user_name}/.m2/settings.xml")
+  fi
+
   if [[ "${DOCKERCACHE}" = true ]]; then
     modp+=("--mvncache=/maven")
   else
@@ -540,14 +557,32 @@ function makearelease
     signflags=("-Psign" "-Dgpg.useagent=true" "-Dgpg.executable=${GPG}")
   fi
 
+  local target="install"
+  if [[ "${DEPLOY}" = true ]]; then
+    target="deploy"
+  fi
+
   # Create SRC and BIN tarballs for release,
   # shellcheck disable=SC2046
-  run_and_redirect "${LOGDIR}/mvn_install.log" \
-    "${MVN}" "${MVN_ARGS[@]}" install \
+  run_and_redirect "${LOGDIR}/mvn_${target}.log" \
+    "${MVN}" "${MVN_ARGS[@]}" ${target} \
       -Pdist,src,yarn-ui \
       "${signflags[@]}" \
       -DskipTests -Dtar $(hadoop_native_flags)
 
+  # Stage BIN tarball
+  run cd "${BASEDIR}"
+  run mv \
+    "${BASEDIR}/hadoop-dist/target/hadoop-${HADOOP_VERSION}.tar.gz" \
+    "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz"
+
+  # Stage SRC tarball
+  run mv \
+    "${BASEDIR}/hadoop-dist/target/hadoop-${HADOOP_VERSION}-src.tar.gz" \
+    "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-src.tar.gz"
+
+  big_console_header "Maven Site"
+
   if [[ "${SECURITYRELEASE}" = true ]]; then
     DOCFLAGS="-Pdocs"
     hadoop_error "WARNING: Skipping automatic changelog and release notes generation due to --security"
@@ -555,7 +590,6 @@ function makearelease
     DOCFLAGS="-Preleasedocs,docs"
   fi
 
-
   # Create site for release
   # we need to do install again so that jdiff and
   # a few other things get registered in the maven
@@ -568,13 +602,17 @@ function makearelease
       -Pdist,src \
       "${DOCFLAGS}"
 
-  big_console_header "Staging the release"
-
+  # Create the site tarball
   run mv "${BASEDIR}/target/staging/hadoop-project" "${BASEDIR}/target/r${HADOOP_VERSION}/"
   run cd "${BASEDIR}/target/"
   run tar czpf "hadoop-site-${HADOOP_VERSION}.tar.gz" "r${HADOOP_VERSION}"/*
   run cd "${BASEDIR}"
 
+  # Stage SITE tarball
+  run mv \
+     "${BASEDIR}/target/hadoop-site-${HADOOP_VERSION}.tar.gz" \
+    "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-site.tar.gz"
+
   # Stage RAT report
   #shellcheck disable=SC2038
   find . -name rat.txt | xargs -I% cat % > "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-rat.txt"
@@ -586,25 +624,14 @@ function makearelease
         "${ARTIFACTS_DIR}/${i}.md"
   done
 
-  # Prepare and stage BIN tarball
-  run cd "${BASEDIR}/hadoop-dist/target/"
-  run tar -xzpf "hadoop-${HADOOP_VERSION}.tar.gz"
+  # We need to fixup the BIN tarball at the end to contain the site docs.
+  run cd "${ARTIFACTS_DIR}"
+  run tar -xzpf "hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz"
+  run mkdir -p "hadoop-${HADOOP_VERSION}/share/doc/hadoop/"
   run cp -r "${BASEDIR}/target/r${HADOOP_VERSION}"/* "hadoop-${HADOOP_VERSION}/share/doc/hadoop/"
-  run tar -czpf "hadoop-${HADOOP_VERSION}.tar.gz" "hadoop-${HADOOP_VERSION}"
-  run cd "${BASEDIR}"
-  run mv \
-    "${BASEDIR}/hadoop-dist/target/hadoop-${HADOOP_VERSION}.tar.gz" \
-    "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz"
-
-  # Stage SRC tarball
-  run mv \
-    "${BASEDIR}/hadoop-dist/target/hadoop-${HADOOP_VERSION}-src.tar.gz" \
-    "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-src.tar.gz"
+  run tar -czpf "hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz" "hadoop-${HADOOP_VERSION}"
+  run rm -rf "hadoop-${HADOOP_VERSION}"
 
-  # Stage SITE tarball
-  run mv \
-     "${BASEDIR}/target/hadoop-site-${HADOOP_VERSION}.tar.gz" \
-    "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-site.tar.gz"
 }
 
 function signartifacts


---------------------------------------------------------------------
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-7438. Additional changes to make SchedulingPlacementSet agnostic to ResourceRequest / placement algorithm. Contributed by Wangda Tan

Posted by ae...@apache.org.
YARN-7438. Additional changes to make SchedulingPlacementSet agnostic to ResourceRequest / placement algorithm. Contributed by Wangda Tan


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

Branch: refs/heads/HDFS-7240
Commit: a957f1c60e1308d1d70a1803381994f59949c5f8
Parents: 3150c01
Author: Sunil G <su...@apache.org>
Authored: Tue Dec 5 22:50:07 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Tue Dec 5 22:50:07 2017 +0530

----------------------------------------------------------------------
 .../rmcontainer/RMContainer.java                |  5 +-
 .../rmcontainer/RMContainerImpl.java            | 20 +++---
 .../scheduler/AbstractYarnScheduler.java        |  7 ++-
 .../scheduler/AppSchedulingInfo.java            | 66 +++++++++-----------
 .../scheduler/ContainerUpdateContext.java       | 11 ++--
 .../scheduler/SchedulerApplicationAttempt.java  | 15 ++---
 .../allocator/RegularContainerAllocator.java    |  3 +-
 .../scheduler/common/ContainerRequest.java      | 54 ++++++++++++++++
 .../scheduler/common/fica/FiCaSchedulerApp.java | 15 ++---
 .../scheduler/fair/FSAppAttempt.java            |  5 +-
 .../scheduler/fifo/FifoAppAttempt.java          |  5 +-
 .../placement/AppPlacementAllocator.java        | 19 +++---
 .../LocalityAppPlacementAllocator.java          | 32 +++++-----
 .../placement/PendingAskUpdateResult.java       | 65 +++++++++++++++++++
 .../placement/ResourceRequestUpdateResult.java  | 43 -------------
 .../server/resourcemanager/Application.java     |  6 +-
 .../rmcontainer/TestRMContainerImpl.java        |  8 +--
 .../capacity/TestCapacityScheduler.java         |  3 +-
 18 files changed, 227 insertions(+), 155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.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/rmcontainer/RMContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
index 29680e5..f3cbf63 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
@@ -31,6 +31,7 @@ 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.event.EventHandler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 
 
@@ -86,8 +87,8 @@ public interface RMContainer extends EventHandler<RMContainerEvent>,
   ContainerReport createContainerReport();
   
   boolean isAMContainer();
-  
-  List<ResourceRequest> getResourceRequests();
+
+  ContainerRequest getContainerRequest();
 
   String getNodeHttpAddress();
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.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/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index a43459c..e26689e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeUpdateContainerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
@@ -178,7 +179,7 @@ public class RMContainerImpl implements RMContainer {
   private long finishTime;
   private ContainerStatus finishedStatus;
   private boolean isAMContainer;
-  private List<ResourceRequest> resourceRequests;
+  private ContainerRequest containerRequestForRecovery;
 
   // Only used for container resource increase and decrease. This is the
   // resource to rollback to should container resource increase token expires.
@@ -233,7 +234,6 @@ public class RMContainerImpl implements RMContainer {
     this.eventHandler = rmContext.getDispatcher().getEventHandler();
     this.containerAllocationExpirer = rmContext.getContainerAllocationExpirer();
     this.isAMContainer = false;
-    this.resourceRequests = null;
     this.nodeLabelExpression = nodeLabelExpression;
     this.lastConfirmedResource = container.getResource();
     this.isExternallyAllocated = isExternallyAllocated;
@@ -412,21 +412,21 @@ public class RMContainerImpl implements RMContainer {
       readLock.unlock();
     }
   }
-  
+
   @Override
-  public List<ResourceRequest> getResourceRequests() {
+  public ContainerRequest getContainerRequest() {
     try {
       readLock.lock();
-      return resourceRequests;
+      return containerRequestForRecovery;
     } finally {
       readLock.unlock();
     }
   }
-  
-  public void setResourceRequests(List<ResourceRequest> requests) {
+
+  public void setContainerRequest(ContainerRequest request) {
+    writeLock.lock();
     try {
-      writeLock.lock();
-      this.resourceRequests = requests;
+      this.containerRequestForRecovery = request;
     } finally {
       writeLock.unlock();
     }
@@ -576,7 +576,7 @@ public class RMContainerImpl implements RMContainer {
     public void transition(RMContainerImpl container, RMContainerEvent event) {
       // Clear ResourceRequest stored in RMContainer, we don't need to remember
       // this anymore.
-      container.setResourceRequests(null);
+      container.setContainerRequest(null);
       
       // Register with containerAllocationExpirer.
       container.containerAllocationExpirer.register(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/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 4749c3d..d94efb1 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
@@ -86,6 +86,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeFinishedContai
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdateEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
 
 
@@ -600,10 +601,10 @@ public abstract class AbstractYarnScheduler
    * @param rmContainer rmContainer
    */
   private void recoverResourceRequestForContainer(RMContainer rmContainer) {
-    List<ResourceRequest> requests = rmContainer.getResourceRequests();
+    ContainerRequest containerRequest = rmContainer.getContainerRequest();
 
     // If container state is moved to ACQUIRED, request will be empty.
-    if (requests == null) {
+    if (containerRequest == null) {
       return;
     }
 
@@ -618,7 +619,7 @@ public abstract class AbstractYarnScheduler
     SchedulerApplicationAttempt schedulerAttempt =
         getCurrentAttemptForContainer(rmContainer.getContainerId());
     if (schedulerAttempt != null) {
-      schedulerAttempt.recoverResourceRequestsForContainer(requests);
+      schedulerAttempt.recoverResourceRequestsForContainer(containerRequest);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
index 9f49880..e47f0c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
@@ -45,10 +45,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.AppPlacementAllocator;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.LocalityAppPlacementAllocator;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.ResourceRequestUpdateResult;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PendingAskUpdateResult;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.util.resource.Resources;
 /**
@@ -220,16 +221,14 @@ public class AppSchedulingInfo {
       }
 
       // Update AppPlacementAllocator
-      ResourceRequestUpdateResult pendingAmountChanges =
+      PendingAskUpdateResult pendingAmountChanges =
           schedulerKeyToAppPlacementAllocator.get(schedulerRequestKey)
-              .updateResourceRequests(
-                  entry.getValue().values(),
+              .updatePendingAsk(entry.getValue().values(),
                   recoverPreemptedRequestForAContainer);
 
       if (null != pendingAmountChanges) {
         updatePendingResources(
-            pendingAmountChanges.getLastAnyResourceRequest(),
-            pendingAmountChanges.getNewResourceRequest(), schedulerRequestKey,
+            pendingAmountChanges, schedulerRequestKey,
             queue.getMetrics());
         offswitchResourcesUpdated = true;
       }
@@ -237,12 +236,17 @@ public class AppSchedulingInfo {
     return offswitchResourcesUpdated;
   }
 
-  private void updatePendingResources(ResourceRequest lastRequest,
-      ResourceRequest request, SchedulerRequestKey schedulerKey,
-      QueueMetrics metrics) {
+  private void updatePendingResources(PendingAskUpdateResult updateResult,
+      SchedulerRequestKey schedulerKey, QueueMetrics metrics) {
+
+    PendingAsk lastPendingAsk = updateResult.getLastPendingAsk();
+    PendingAsk newPendingAsk = updateResult.getNewPendingAsk();
+    String lastNodePartition = updateResult.getLastNodePartition();
+    String newNodePartition = updateResult.getNewNodePartition();
+
     int lastRequestContainers =
-        (lastRequest != null) ? lastRequest.getNumContainers() : 0;
-    if (request.getNumContainers() <= 0) {
+        (lastPendingAsk != null) ? lastPendingAsk.getCount() : 0;
+    if (newPendingAsk.getCount() <= 0) {
       if (lastRequestContainers >= 0) {
         schedulerKeys.remove(schedulerKey);
         schedulerKeyToAppPlacementAllocator.remove(schedulerKey);
@@ -258,31 +262,23 @@ public class AppSchedulingInfo {
       }
     }
 
-    Resource lastRequestCapability =
-        lastRequest != null ? lastRequest.getCapability() : Resources.none();
-    metrics.incrPendingResources(request.getNodeLabelExpression(), user,
-        request.getNumContainers(), request.getCapability());
-
-    if(lastRequest != null) {
-      metrics.decrPendingResources(lastRequest.getNodeLabelExpression(), user,
-          lastRequestContainers, lastRequestCapability);
+    if (lastPendingAsk != null) {
+      // Deduct resources from metrics / pending resources of queue/app.
+      metrics.decrPendingResources(lastNodePartition, user,
+          lastPendingAsk.getCount(), lastPendingAsk.getPerAllocationResource());
+      Resource decreasedResource = Resources.multiply(
+          lastPendingAsk.getPerAllocationResource(), lastRequestContainers);
+      queue.decPendingResource(lastNodePartition, decreasedResource);
+      appResourceUsage.decPending(lastNodePartition, decreasedResource);
     }
 
-    // update queue:
-    Resource increasedResource =
-        Resources.multiply(request.getCapability(), request.getNumContainers());
-    queue.incPendingResource(request.getNodeLabelExpression(),
-        increasedResource);
-    appResourceUsage.incPending(request.getNodeLabelExpression(),
-        increasedResource);
-    if (lastRequest != null) {
-      Resource decreasedResource =
-          Resources.multiply(lastRequestCapability, lastRequestContainers);
-      queue.decPendingResource(lastRequest.getNodeLabelExpression(),
-          decreasedResource);
-      appResourceUsage.decPending(lastRequest.getNodeLabelExpression(),
-          decreasedResource);
-    }
+    // Increase resources to metrics / pending resources of queue/app.
+    metrics.incrPendingResources(newNodePartition, user,
+        newPendingAsk.getCount(), newPendingAsk.getPerAllocationResource());
+    Resource increasedResource = Resources.multiply(
+        newPendingAsk.getPerAllocationResource(), newPendingAsk.getCount());
+    queue.incPendingResource(newNodePartition, increasedResource);
+    appResourceUsage.incPending(newNodePartition, increasedResource);
   }
 
   public void addRequestedPartition(String partition) {
@@ -417,7 +413,7 @@ public class AppSchedulingInfo {
     }
   }
 
-  public List<ResourceRequest> allocate(NodeType type,
+  public ContainerRequest allocate(NodeType type,
       SchedulerNode node, SchedulerRequestKey schedulerKey,
       Container containerAllocated) {
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerUpdateContext.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/ContainerUpdateContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerUpdateContext.java
index 93995a1..f410db1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerUpdateContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerUpdateContext.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer
     .RMContainerImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.AppPlacementAllocator;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -155,16 +156,16 @@ public class ContainerUpdateContext {
     AppPlacementAllocator<SchedulerNode> appPlacementAllocator =
         appSchedulingInfo.getAppPlacementAllocator(schedulerKey);
     if (appPlacementAllocator != null) {
-      Map<String, ResourceRequest> resourceRequests = appPlacementAllocator
-          .getResourceRequests();
-      ResourceRequest prevReq = resourceRequests.get(ResourceRequest.ANY);
+      PendingAsk pendingAsk = appPlacementAllocator.getPendingAsk(
+          ResourceRequest.ANY);
       // Decrement the pending using a dummy RR with
       // resource = prev update req capability
-      if (prevReq != null) {
+      if (pendingAsk != null && pendingAsk.getCount() > 0) {
         appSchedulingInfo.allocate(NodeType.OFF_SWITCH, schedulerNode,
             schedulerKey, Container.newInstance(UNDEFINED,
                 schedulerNode.getNodeID(), "host:port",
-                prevReq.getCapability(), schedulerKey.getPriority(), null));
+                pendingAsk.getPerAllocationResource(),
+                schedulerKey.getPriority(), null));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index 65673c9..dfb0e67 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -76,6 +76,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerUpda
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeUpdateContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.AppPlacementAllocator;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.SchedulableEntity;
@@ -449,11 +450,12 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
   }
   
   public void recoverResourceRequestsForContainer(
-      List<ResourceRequest> requests) {
+      ContainerRequest containerRequest) {
     try {
       writeLock.lock();
       if (!isStopped) {
-        appSchedulingInfo.updateResourceRequests(requests, true);
+        appSchedulingInfo.updateResourceRequests(
+            containerRequest.getResourceRequests(), true);
       }
     } finally {
       writeLock.unlock();
@@ -913,7 +915,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
         RMContainer c = tempIter.next();
         // Mark container for release (set RRs to null, so RM does not think
         // it is a recoverable container)
-        ((RMContainerImpl) c).setResourceRequests(null);
+        ((RMContainerImpl) c).setContainerRequest(null);
 
         // Release this container async-ly so as to prevent
         // 'LeafQueue::completedContainer()' from trying to acquire a lock
@@ -1383,13 +1385,6 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
       SchedulerRequestKey schedulerRequestKey) {
     return appSchedulingInfo.getAppPlacementAllocator(schedulerRequestKey);
   }
-
-  public Map<String, ResourceRequest> getResourceRequests(
-      SchedulerRequestKey schedulerRequestKey) {
-    return appSchedulingInfo.getAppPlacementAllocator(schedulerRequestKey)
-        .getResourceRequests();
-  }
-
   public void incUnconfirmedRes(Resource res) {
     unconfirmedAllocatedMem.addAndGet(res.getMemorySize());
     unconfirmedAllocatedVcores.addAndGet(res.getVirtualCores());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
index 69e90c6..2642532 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
@@ -299,7 +299,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
       }
       // If we have only ANY requests for this schedulerKey, we should not
       // delay its scheduling.
-      if (application.getResourceRequests(schedulerKey).size() == 1) {
+      if (application.getAppPlacementAllocator(schedulerKey)
+          .getUniqueLocationAsks() == 1) {
         return true;
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ContainerRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ContainerRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ContainerRequest.java
new file mode 100644
index 0000000..075db79
--- /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/common/ContainerRequest.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common;
+
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+
+import java.util.List;
+
+/**
+ * ContainerRequest is a class to capture resource requests associated with a
+ * Container, this will be used by scheduler to recover resource requests if the
+ * container preempted or cancelled before AM acquire the container.
+ *
+ * It should include deducted resource requests when the container allocated.
+ *
+ * Lifecycle of the ContainerRequest is:
+ *
+ * <pre>
+ * 1) It is instantiated when container created.
+ * 2) It will be set to ContainerImpl by scheduler.
+ * 3) When container preempted or cancelled because of whatever reason before
+ *    container acquired by AM. ContainerRequest will be added back to pending
+ *    request pool.
+ * 4) It will be cleared from ContainerImpl if the container already acquired by
+ *    AM.
+ * </pre>
+ */
+public class ContainerRequest {
+  private List<ResourceRequest> requests;
+
+  public ContainerRequest(List<ResourceRequest> requests) {
+    this.requests = requests;
+  }
+
+  public List<ResourceRequest> getResourceRequests() {
+    return requests;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
index 93d51d8..34594cf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
@@ -73,6 +73,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Scheduli
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.AbstractContainerAllocator;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.ContainerAllocator;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.SchedulerContainer;
@@ -369,7 +370,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
   public boolean accept(Resource cluster,
       ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request) {
-    List<ResourceRequest> resourceRequests = null;
+    ContainerRequest containerRequest = null;
     boolean reReservation = false;
 
     try {
@@ -397,8 +398,8 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
         if (schedulerContainer.isAllocated()) {
           // When allocate a new container
-          resourceRequests =
-              schedulerContainer.getRmContainer().getResourceRequests();
+          containerRequest =
+              schedulerContainer.getRmContainer().getContainerRequest();
 
           // Check pending resource request
           if (!appSchedulingInfo.checkAllocation(allocation.getAllocationLocalityType(),
@@ -471,8 +472,8 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     }
 
     // When rejected, recover resource requests for this app
-    if (!accepted && resourceRequests != null) {
-      recoverResourceRequestsForContainer(resourceRequests);
+    if (!accepted && containerRequest != null) {
+      recoverResourceRequestsForContainer(containerRequest);
     }
 
     return accepted;
@@ -524,12 +525,12 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
           liveContainers.put(containerId, rmContainer);
 
           // Deduct pending resource requests
-          List<ResourceRequest> requests = appSchedulingInfo.allocate(
+          ContainerRequest containerRequest = appSchedulingInfo.allocate(
               allocation.getAllocationLocalityType(),
               schedulerContainer.getSchedulerNode(),
               schedulerContainer.getSchedulerRequestKey(),
               schedulerContainer.getRmContainer().getContainer());
-          ((RMContainerImpl) rmContainer).setResourceRequests(requests);
+          ((RMContainerImpl) rmContainer).setContainerRequest(containerRequest);
 
           attemptResourceUsage.incUsed(schedulerContainer.getNodePartition(),
               allocation.getAllocatedOrReservedResource());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/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 43daace..e095a42 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
@@ -55,6 +55,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
@@ -460,13 +461,13 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
       liveContainers.put(container.getId(), rmContainer);
 
       // Update consumption and track allocations
-      List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
+      ContainerRequest containerRequest = appSchedulingInfo.allocate(
           type, node, schedulerKey, container);
       this.attemptResourceUsage.incUsed(container.getResource());
       getQueue().incUsedResource(container.getResource());
 
       // Update resource requests related to "request" and store in RMContainer
-      ((RMContainerImpl) rmContainer).setResourceRequests(resourceRequestList);
+      ((RMContainerImpl) rmContainer).setContainerRequest(containerRequest);
 
       // Inform the container
       rmContainer.handle(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java
index d932e0e..169b98a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 
@@ -80,14 +81,14 @@ public class FifoAppAttempt extends FiCaSchedulerApp {
       liveContainers.put(containerId, rmContainer);
 
       // Update consumption and track allocations
-      List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
+      ContainerRequest containerRequest = appSchedulingInfo.allocate(
           type, node, schedulerKey, container);
 
       attemptResourceUsage.incUsed(node.getPartition(),
           container.getResource());
 
       // Update resource requests related to "request" and store in RMContainer
-      ((RMContainerImpl) rmContainer).setResourceRequests(resourceRequestList);
+      ((RMContainerImpl) rmContainer).setContainerRequest(containerRequest);
 
       // Inform the container
       rmContainer.handle(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.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/placement/AppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java
index 63b22a3..dcb38aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 
@@ -57,14 +58,14 @@ public interface AppPlacementAllocator<N extends SchedulerNode> {
   Iterator<N> getPreferredNodeIterator(CandidateNodeSet<N> candidateNodeSet);
 
   /**
-   * Replace existing ResourceRequest by the new requests
+   * Replace existing pending asks by the new requests
    *
-   * @param requests new ResourceRequests
+   * @param requests new asks
    * @param recoverPreemptedRequestForAContainer if we're recovering resource
    * requests for preempted container
    * @return true if total pending resource changed
    */
-  ResourceRequestUpdateResult updateResourceRequests(
+  PendingAskUpdateResult updatePendingAsk(
       Collection<ResourceRequest> requests,
       boolean recoverPreemptedRequestForAContainer);
 
@@ -97,18 +98,14 @@ public interface AppPlacementAllocator<N extends SchedulerNode> {
    * @param schedulerKey SchedulerRequestKey for this ResourceRequest
    * @param type Type of the allocation
    * @param node Which node this container allocated on
-   * @return list of ResourceRequests deducted
+   * @return ContainerRequest which include resource requests associated with
+   *         the container. This will be used by scheduler to recover requests.
+   *         Please refer to {@link ContainerRequest} for more details.
    */
-  List<ResourceRequest> allocate(SchedulerRequestKey schedulerKey,
+  ContainerRequest allocate(SchedulerRequestKey schedulerKey,
       NodeType type, SchedulerNode node);
 
   /**
-   * Returns list of accepted resourceNames.
-   * @return Iterator of accepted resourceNames
-   */
-  Iterator<String> getAcceptedResouceNames();
-
-  /**
    * We can still have pending requirement for a given NodeType and node
    * @param type Locality Type
    * @param node which node we will allocate on

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.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/placement/LocalityAppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
index 7f89435..766827c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 
@@ -44,7 +45,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
  * into account locality preferences (node, rack, any) when allocating
  * containers.
  */
-public class LocalityAppPlacementAllocator<N extends SchedulerNode>
+public class LocalityAppPlacementAllocator <N extends SchedulerNode>
     implements AppPlacementAllocator<N> {
   private static final Log LOG =
       LogFactory.getLog(LocalityAppPlacementAllocator.class);
@@ -122,13 +123,13 @@ public class LocalityAppPlacementAllocator<N extends SchedulerNode>
   }
 
   @Override
-  public ResourceRequestUpdateResult updateResourceRequests(
+  public PendingAskUpdateResult updatePendingAsk(
       Collection<ResourceRequest> requests,
       boolean recoverPreemptedRequestForAContainer) {
     try {
       this.writeLock.lock();
 
-      ResourceRequestUpdateResult updateResult = null;
+      PendingAskUpdateResult updateResult = null;
 
       // Update resource requests
       for (ResourceRequest request : requests) {
@@ -156,7 +157,16 @@ public class LocalityAppPlacementAllocator<N extends SchedulerNode>
           //update the applications requested labels set
           appSchedulingInfo.addRequestedPartition(partition);
 
-          updateResult = new ResourceRequestUpdateResult(lastRequest, request);
+          PendingAsk lastPendingAsk =
+              lastRequest == null ? null : new PendingAsk(
+                  lastRequest.getCapability(), lastRequest.getNumContainers());
+          String lastRequestedNodePartition =
+              lastRequest == null ? null : lastRequest.getNodeLabelExpression();
+
+          updateResult = new PendingAskUpdateResult(lastPendingAsk,
+              new PendingAsk(request.getCapability(),
+                  request.getNumContainers()), lastRequestedNodePartition,
+              request.getNodeLabelExpression());
         }
       }
       return updateResult;
@@ -380,7 +390,7 @@ public class LocalityAppPlacementAllocator<N extends SchedulerNode>
   }
 
   @Override
-  public List<ResourceRequest> allocate(SchedulerRequestKey schedulerKey,
+  public ContainerRequest allocate(SchedulerRequestKey schedulerKey,
       NodeType type, SchedulerNode node) {
     try {
       writeLock.lock();
@@ -404,19 +414,9 @@ public class LocalityAppPlacementAllocator<N extends SchedulerNode>
         allocateOffSwitch(schedulerKey, request, resourceRequests);
       }
 
-      return resourceRequests;
+      return new ContainerRequest(resourceRequests);
     } finally {
       writeLock.unlock();
     }
   }
-
-  @Override
-  public Iterator<String> getAcceptedResouceNames() {
-    try {
-      readLock.lock();
-      return resourceRequestMap.keySet().iterator();
-    } finally {
-      readLock.unlock();
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/PendingAskUpdateResult.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/placement/PendingAskUpdateResult.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/PendingAskUpdateResult.java
new file mode 100644
index 0000000..8765e86
--- /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/placement/PendingAskUpdateResult.java
@@ -0,0 +1,65 @@
+/**
+ * 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.placement;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
+
+/**
+ * Result of a resource-request update. This will be used by
+ * {@link org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo}
+ * to update queue metrics and application/queue's overall pending resources.
+ * And this is per-scheduler-key.
+ *
+ * Following fields will be set if pending ask changed for a given scheduler key
+ * - lastPendingAsk: how many resource asked before.
+ * - newPendingAsk: how many resource asked now.
+ * - lastNodePartition: what's the node partition before.
+ * - newNodePartition: what's the node partition now.
+ */
+public class PendingAskUpdateResult {
+  private final PendingAsk lastPendingAsk;
+  private final String lastNodePartition;
+  private final PendingAsk newPendingAsk;
+  private final String newNodePartition;
+
+  public PendingAskUpdateResult(PendingAsk lastPendingAsk,
+      PendingAsk newPendingAsk, String lastNodePartition,
+      String newNodePartition) {
+    this.lastPendingAsk = lastPendingAsk;
+    this.newPendingAsk = newPendingAsk;
+    this.lastNodePartition = lastNodePartition;
+    this.newNodePartition = newNodePartition;
+  }
+
+  public PendingAsk getLastPendingAsk() {
+    return lastPendingAsk;
+  }
+
+  public PendingAsk getNewPendingAsk() {
+    return newPendingAsk;
+  }
+
+  public String getLastNodePartition() {
+    return lastNodePartition;
+  }
+
+  public String getNewNodePartition() {
+    return newNodePartition;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/ResourceRequestUpdateResult.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/placement/ResourceRequestUpdateResult.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/ResourceRequestUpdateResult.java
deleted file mode 100644
index da356f5..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/ResourceRequestUpdateResult.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
-
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
-
-/**
- * Result of ResourceRequest update
- */
-public class ResourceRequestUpdateResult {
-  private final ResourceRequest lastAnyResourceRequest;
-  private final ResourceRequest newResourceRequest;
-
-  public ResourceRequestUpdateResult(ResourceRequest lastAnyResourceRequest,
-      ResourceRequest newResourceRequest) {
-    this.lastAnyResourceRequest = lastAnyResourceRequest;
-    this.newResourceRequest = newResourceRequest;
-  }
-
-  public ResourceRequest getLastAnyResourceRequest() {
-    return lastAnyResourceRequest;
-  }
-
-  public ResourceRequest getNewResourceRequest() {
-    return newResourceRequest;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.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/Application.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
index e889de0..fbde681 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
@@ -431,7 +431,7 @@ public class Application {
     if (type == NodeType.NODE_LOCAL) {
       for (String host : task.getHosts()) {
         if(LOG.isDebugEnabled()) {
-          LOG.debug("updateResourceRequests:" + " application=" + applicationId
+          LOG.debug("updatePendingAsk:" + " application=" + applicationId
             + " type=" + type + " host=" + host
             + " request=" + ((requests == null) ? "null" : requests.get(host)));
         }
@@ -442,7 +442,7 @@ public class Application {
     if (type == NodeType.NODE_LOCAL || type == NodeType.RACK_LOCAL) {
       for (String rack : task.getRacks()) {
         if(LOG.isDebugEnabled()) {
-          LOG.debug("updateResourceRequests:" + " application=" + applicationId
+          LOG.debug("updatePendingAsk:" + " application=" + applicationId
             + " type=" + type + " rack=" + rack
             + " request=" + ((requests == null) ? "null" : requests.get(rack)));
         }
@@ -453,7 +453,7 @@ public class Application {
     updateResourceRequest(requests.get(ResourceRequest.ANY));
     
     if(LOG.isDebugEnabled()) {
-      LOG.debug("updateResourceRequests:" + " application=" + applicationId
+      LOG.debug("updatePendingAsk:" + " application=" + applicationId
         + " #asks=" + ask.size());
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.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/rmcontainer/TestRMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
index db31448..6c189b3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
@@ -278,8 +278,8 @@ public class TestRMContainerImpl {
 
     // Verify whether list of ResourceRequest is present in RMContainer
     // while moving to ALLOCATED state
-    Assert.assertNotNull(scheduler.getRMContainer(containerId2)
-        .getResourceRequests());
+    Assert.assertNotNull(
+        scheduler.getRMContainer(containerId2).getContainerRequest());
 
     // Allocate container
     am1.allocate(new ArrayList<ResourceRequest>(), new ArrayList<ContainerId>())
@@ -288,8 +288,8 @@ public class TestRMContainerImpl {
 
     // After RMContainer moving to ACQUIRED state, list of ResourceRequest will
     // be empty
-    Assert.assertNull(scheduler.getRMContainer(containerId2)
-        .getResourceRequests());
+    Assert.assertNull(
+        scheduler.getRMContainer(containerId2).getContainerRequest());
   }
 
   @Test (timeout = 180000)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a957f1c6/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 1edb0da..e91f734 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
@@ -1696,7 +1696,8 @@ public class TestCapacityScheduler {
     rm1.waitForState(nm1, containerId1, RMContainerState.ALLOCATED);
 
     RMContainer rmContainer = cs.getRMContainer(containerId1);
-    List<ResourceRequest> requests = rmContainer.getResourceRequests();
+    List<ResourceRequest> requests =
+        rmContainer.getContainerRequest().getResourceRequests();
     FiCaSchedulerApp app = cs.getApplicationAttempt(am1
         .getApplicationAttemptId());
 


---------------------------------------------------------------------
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-7587. Skip dispatching opportunistic containers to nodes whose queue is already full. (Weiwei Yang via asuresh)

Posted by ae...@apache.org.
YARN-7587. Skip dispatching opportunistic containers to nodes whose queue is already full. (Weiwei Yang via asuresh)


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

Branch: refs/heads/HDFS-7240
Commit: 37ca4169508c3003dbe9044fefd37eb8cd8c0503
Parents: 81f6e46
Author: Arun Suresh <as...@apache.org>
Authored: Sun Dec 3 22:22:01 2017 -0800
Committer: Arun Suresh <as...@apache.org>
Committed: Sun Dec 3 22:22:01 2017 -0800

----------------------------------------------------------------------
 .../records/OpportunisticContainersStatus.java  | 19 +++++++++++++++++
 .../pb/OpportunisticContainersStatusPBImpl.java | 13 ++++++++++++
 .../main/proto/yarn_server_common_protos.proto  |  1 +
 .../scheduler/ContainerScheduler.java           | 12 +++++++++++
 .../distributed/NodeQueueLoadMonitor.java       | 22 ++++++++++++++++++--
 .../distributed/TestNodeQueueLoadMonitor.java   | 21 +++++++++++++++++++
 6 files changed, 86 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/37ca4169/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/OpportunisticContainersStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/OpportunisticContainersStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/OpportunisticContainersStatus.java
index 732db2a..c8a81a7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/OpportunisticContainersStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/OpportunisticContainersStatus.java
@@ -149,4 +149,23 @@ public abstract class OpportunisticContainersStatus {
   @Unstable
   public abstract void setEstimatedQueueWaitTime(int queueWaitTime);
 
+
+  /**
+   * Gets the capacity of the opportunistic containers queue on the node.
+   *
+   * @return queue capacity.
+   */
+  @Private
+  @Unstable
+  public abstract int getOpportQueueCapacity();
+
+
+  /**
+   * Sets the capacity of the opportunistic containers queue on the node.
+   *
+   * @param queueCapacity queue capacity.
+   */
+  @Private
+  @Unstable
+  public abstract void setOpportQueueCapacity(int queueCapacity);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37ca4169/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/OpportunisticContainersStatusPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/OpportunisticContainersStatusPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/OpportunisticContainersStatusPBImpl.java
index 8399713..5d1005c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/OpportunisticContainersStatusPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/OpportunisticContainersStatusPBImpl.java
@@ -136,4 +136,17 @@ public class OpportunisticContainersStatusPBImpl
     maybeInitBuilder();
     builder.setEstimatedQueueWaitTime(queueWaitTime);
   }
+
+  @Override
+  public int getOpportQueueCapacity() {
+    YarnServerCommonProtos.OpportunisticContainersStatusProtoOrBuilder p =
+        viaProto ? proto : builder;
+    return p.getOpportQueueCapacity();
+  }
+
+  @Override
+  public void setOpportQueueCapacity(int maxOpportQueueLength) {
+    maybeInitBuilder();
+    builder.setOpportQueueCapacity(maxOpportQueueLength);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37ca4169/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto
index 98b172d..8200808 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto
@@ -49,6 +49,7 @@ message OpportunisticContainersStatusProto {
   optional int32 queued_opport_containers = 4;
   optional int32 wait_queue_length = 5;
   optional int32 estimated_queue_wait_time = 6;
+  optional int32 opport_queue_capacity = 7;
 }
 
 message MasterKeyProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37ca4169/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
index 76da37c..d9b713f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
@@ -68,6 +68,7 @@ public class ContainerScheduler extends AbstractService implements
       LoggerFactory.getLogger(ContainerScheduler.class);
 
   private final Context context;
+  // Capacity of the queue for opportunistic Containers.
   private final int maxOppQueueLength;
 
   // Queue of Guaranteed Containers waiting for resources to run
@@ -258,6 +259,15 @@ public class ContainerScheduler extends AbstractService implements
         + this.queuedOpportunisticContainers.size();
   }
 
+  /**
+   * Return the capacity of the queue for opportunistic containers
+   * on this node.
+   * @return queue capacity.
+   */
+  public int getOpportunisticQueueCapacity() {
+    return this.maxOppQueueLength;
+  }
+
   @VisibleForTesting
   public int getNumQueuedGuaranteedContainers() {
     return this.queuedGuaranteedContainers.size();
@@ -290,6 +300,8 @@ public class ContainerScheduler extends AbstractService implements
         metrics.getAllocatedOpportunisticVCores());
     this.opportunisticContainersStatus.setRunningOpportContainers(
         metrics.getRunningOpportunisticContainers());
+    this.opportunisticContainersStatus.setOpportQueueCapacity(
+        getOpportunisticQueueCapacity());
     return this.opportunisticContainersStatus;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37ca4169/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.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/distributed/NodeQueueLoadMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.java
index ed0ee1e..e989099 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.java
@@ -75,6 +75,7 @@ public class NodeQueueLoadMonitor implements ClusterMonitor {
     int queueWaitTime = -1;
     double timestamp;
     final NodeId nodeId;
+    private int queueCapacity = 0;
 
     public ClusterNode(NodeId nodeId) {
       this.nodeId = nodeId;
@@ -95,6 +96,16 @@ public class NodeQueueLoadMonitor implements ClusterMonitor {
       this.timestamp = System.currentTimeMillis();
       return this;
     }
+
+    public ClusterNode setQueueCapacity(int capacity) {
+      this.queueCapacity = capacity;
+      return this;
+    }
+
+    public boolean isQueueFull() {
+      return this.queueCapacity > 0 &&
+          this.queueLength >= this.queueCapacity;
+    }
   }
 
   private final ScheduledExecutorService scheduledExecutor;
@@ -207,6 +218,8 @@ public class NodeQueueLoadMonitor implements ClusterMonitor {
       opportunisticContainersStatus =
           OpportunisticContainersStatus.newInstance();
     }
+    int opportQueueCapacity =
+        opportunisticContainersStatus.getOpportQueueCapacity();
     int estimatedQueueWaitTime =
         opportunisticContainersStatus.getEstimatedQueueWaitTime();
     int waitQueueLength = opportunisticContainersStatus.getWaitQueueLength();
@@ -222,7 +235,8 @@ public class NodeQueueLoadMonitor implements ClusterMonitor {
           this.clusterNodes.put(rmNode.getNodeID(),
               new ClusterNode(rmNode.getNodeID())
                   .setQueueWaitTime(estimatedQueueWaitTime)
-                  .setQueueLength(waitQueueLength));
+                  .setQueueLength(waitQueueLength)
+                  .setQueueCapacity(opportQueueCapacity));
           LOG.info("Inserting ClusterNode [" + rmNode.getNodeID() + "] " +
               "with queue wait time [" + estimatedQueueWaitTime + "] and " +
               "wait queue length [" + waitQueueLength + "]");
@@ -301,7 +315,11 @@ public class NodeQueueLoadMonitor implements ClusterMonitor {
       // is what we ultimately care about.
       Arrays.sort(nodes, (Comparator)comparator);
       for (int j=0; j < nodes.length; j++) {
-        retList.add(((ClusterNode)nodes[j]).nodeId);
+        ClusterNode cNode = (ClusterNode)nodes[j];
+        // Exclude nodes whose queue is already full.
+        if (!cNode.isQueueFull()) {
+          retList.add(cNode.nodeId);
+        }
       }
       return retList;
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/37ca4169/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/TestNodeQueueLoadMonitor.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/distributed/TestNodeQueueLoadMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/TestNodeQueueLoadMonitor.java
index dfd21ff..85eddaa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/TestNodeQueueLoadMonitor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/TestNodeQueueLoadMonitor.java
@@ -33,6 +33,8 @@ import java.util.List;
  */
 public class TestNodeQueueLoadMonitor {
 
+  private final static int DEFAULT_MAX_QUEUE_LENGTH = 200;
+
   static class FakeNodeId extends NodeId {
     final String host;
     final int port;
@@ -132,6 +134,17 @@ public class TestNodeQueueLoadMonitor {
     Assert.assertEquals("h2:2", nodeIds.get(1).toString());
     Assert.assertEquals("h1:1", nodeIds.get(2).toString());
     Assert.assertEquals("h4:4", nodeIds.get(3).toString());
+
+    // Now update h3 and fill its queue.
+    selector.updateNode(createRMNode("h3", 3, -1,
+        DEFAULT_MAX_QUEUE_LENGTH));
+    selector.computeTask.run();
+    nodeIds = selector.selectNodes();
+    System.out.println("4-> "+ nodeIds);
+    Assert.assertEquals(3, nodeIds.size());
+    Assert.assertEquals("h2:2", nodeIds.get(0).toString());
+    Assert.assertEquals("h1:1", nodeIds.get(1).toString());
+    Assert.assertEquals("h4:4", nodeIds.get(2).toString());
   }
 
   @Test
@@ -180,6 +193,12 @@ public class TestNodeQueueLoadMonitor {
 
   private RMNode createRMNode(String host, int port,
       int waitTime, int queueLength) {
+    return createRMNode(host, port, waitTime, queueLength,
+        DEFAULT_MAX_QUEUE_LENGTH);
+  }
+
+  private RMNode createRMNode(String host, int port,
+      int waitTime, int queueLength, int queueCapacity) {
     RMNode node1 = Mockito.mock(RMNode.class);
     NodeId nID1 = new FakeNodeId(host, port);
     Mockito.when(node1.getNodeID()).thenReturn(nID1);
@@ -189,6 +208,8 @@ public class TestNodeQueueLoadMonitor {
         .thenReturn(waitTime);
     Mockito.when(status1.getWaitQueueLength())
         .thenReturn(queueLength);
+    Mockito.when(status1.getOpportQueueCapacity())
+        .thenReturn(queueCapacity);
     Mockito.when(node1.getOpportunisticContainersStatus()).thenReturn(status1);
     return node1;
   }


---------------------------------------------------------------------
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-12681. Make HdfsLocatedFileStatus a subtype of LocatedFileStatus

Posted by ae...@apache.org.
HDFS-12681. Make HdfsLocatedFileStatus a subtype of LocatedFileStatus


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

Branch: refs/heads/HDFS-7240
Commit: 0e560f3b8d194c10dce06443979df4074e14b0db
Parents: c9a54aa
Author: Chris Douglas <cd...@apache.org>
Authored: Wed Nov 29 20:28:06 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Wed Nov 29 20:28:06 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/FileStatus.java   |  61 ++-
 .../org/apache/hadoop/fs/LocatedFileStatus.java |  42 +-
 .../apache/hadoop/fs/protocolPB/PBHelper.java   |  11 +-
 .../dev-support/findbugsExcludeFile.xml         |  30 ++
 .../hadoop/hdfs/protocol/HdfsFileStatus.java    | 530 ++++++++++---------
 .../hdfs/protocol/HdfsLocatedFileStatus.java    | 208 ++++++--
 .../hdfs/protocol/HdfsNamedFileStatus.java      | 180 +++++++
 .../hadoop/hdfs/protocol/package-info.java      |  18 +
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |  57 +-
 .../protocol/TestHdfsFileStatusMethods.java     | 106 ++++
 .../hadoop/fs/http/client/HttpFSFileSystem.java |   5 +-
 .../dev-support/findbugsExcludeFile.xml         |   7 -
 .../apache/hadoop/hdfs/server/mover/Mover.java  |   2 +-
 .../server/namenode/FSDirStatAndListingOp.java  |  47 +-
 .../hadoop/hdfs/TestBlockStoragePolicy.java     |   2 +-
 .../hdfs/TestFileStatusSerialization.java       |   8 +-
 .../hdfs/server/mover/TestStorageMover.java     |   2 +-
 17 files changed, 925 insertions(+), 391 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e560f3b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
index d7c05d7..0663c43 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
@@ -56,15 +56,36 @@ public class FileStatus implements Writable, Comparable<Object>,
   private Path symlink;
   private Set<AttrFlags> attr;
 
-  private enum AttrFlags {
+  /**
+   * Flags for entity attributes.
+   */
+  public enum AttrFlags {
+    /** ACL information available for this entity. */
     HAS_ACL,
+    /** Entity is encrypted. */
     HAS_CRYPT,
+    /** Entity is stored erasure-coded. */
     HAS_EC,
-    SNAPSHOT_ENABLED
+    /** Snapshot capability enabled. */
+    SNAPSHOT_ENABLED,
   }
-  private static final Set<AttrFlags> NONE = Collections.<AttrFlags>emptySet();
-  private static Set<AttrFlags> flags(boolean acl, boolean crypt, boolean ec) {
-    if (!(acl || crypt || ec)) {
+
+  /**
+   * Shared, empty set of attributes (a common case for FileStatus).
+   */
+  public static final Set<AttrFlags> NONE = Collections.<AttrFlags>emptySet();
+
+  /**
+   * Convert boolean attributes to a set of flags.
+   * @param acl   See {@link AttrFlags#HAS_ACL}.
+   * @param crypt See {@link AttrFlags#HAS_CRYPT}.
+   * @param ec    See {@link AttrFlags#HAS_EC}.
+   * @param sn    See {@link AttrFlags#SNAPSHOT_ENABLED}.
+   * @return converted set of flags.
+   */
+  public static Set<AttrFlags> attributes(boolean acl, boolean crypt,
+                                          boolean ec, boolean sn) {
+    if (!(acl || crypt || ec || sn)) {
       return NONE;
     }
     EnumSet<AttrFlags> ret = EnumSet.noneOf(AttrFlags.class);
@@ -77,6 +98,9 @@ public class FileStatus implements Writable, Comparable<Object>,
     if (ec) {
       ret.add(AttrFlags.HAS_EC);
     }
+    if (sn) {
+      ret.add(AttrFlags.SNAPSHOT_ENABLED);
+    }
     return ret;
   }
 
@@ -117,6 +141,15 @@ public class FileStatus implements Writable, Comparable<Object>,
       long blocksize, long modification_time, long access_time,
       FsPermission permission, String owner, String group, Path symlink,
       Path path, boolean hasAcl, boolean isEncrypted, boolean isErasureCoded) {
+    this(length, isdir, block_replication, blocksize, modification_time,
+        access_time, permission, owner, group, symlink, path,
+        attributes(hasAcl, isEncrypted, isErasureCoded, false));
+  }
+
+  public FileStatus(long length, boolean isdir, int block_replication,
+      long blocksize, long modification_time, long access_time,
+      FsPermission permission, String owner, String group, Path symlink,
+      Path path, Set<AttrFlags> attr) {
     this.length = length;
     this.isdir = isdir;
     this.block_replication = (short)block_replication;
@@ -136,7 +169,7 @@ public class FileStatus implements Writable, Comparable<Object>,
     this.group = (group == null) ? "" : group;
     this.symlink = symlink;
     this.path = path;
-    attr = flags(hasAcl, isEncrypted, isErasureCoded);
+    this.attr = attr;
 
     // The variables isdir and symlink indicate the type:
     // 1. isdir implies directory, in which case symlink must be null.
@@ -341,19 +374,6 @@ public class FileStatus implements Writable, Comparable<Object>,
   }
 
   /**
-   * Sets Snapshot enabled flag.
-   *
-   * @param isSnapShotEnabled When true, SNAPSHOT_ENABLED flag is set
-   */
-  public void setSnapShotEnabledFlag(boolean isSnapShotEnabled) {
-    if (isSnapShotEnabled) {
-      attr.add(AttrFlags.SNAPSHOT_ENABLED);
-    } else {
-      attr.remove(AttrFlags.SNAPSHOT_ENABLED);
-    }
-  }
-
-  /**
    * @return The contents of the symbolic link.
    */
   public Path getSymlink() throws IOException {
@@ -480,7 +500,8 @@ public class FileStatus implements Writable, Comparable<Object>,
     setGroup(other.getGroup());
     setSymlink((other.isSymlink() ? other.getSymlink() : null));
     setPath(other.getPath());
-    attr = flags(other.hasAcl(), other.isEncrypted(), other.isErasureCoded());
+    attr = attributes(other.hasAcl(), other.isEncrypted(),
+        other.isErasureCoded(), other.isSnapshotEnabled());
     assert (isDirectory() && getSymlink() == null) || !isDirectory();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e560f3b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocatedFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocatedFileStatus.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocatedFileStatus.java
index 29e1998..4994957 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocatedFileStatus.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocatedFileStatus.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs;
 
 import java.io.IOException;
+import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -115,9 +116,35 @@ public class LocatedFileStatus extends FileStatus {
       Path symlink, Path path,
       boolean hasAcl, boolean isEncrypted, boolean isErasureCoded,
       BlockLocation[] locations) {
-    super(length, isdir, block_replication, blocksize, modification_time,
+    this(length, isdir, block_replication, blocksize, modification_time,
         access_time, permission, owner, group, symlink, path,
-        hasAcl, isEncrypted, isErasureCoded);
+        attributes(hasAcl, isEncrypted, isErasureCoded, false), locations);
+    this.locations = locations;
+  }
+
+  /**
+   * Constructor.
+   *
+   * @param length a file's length
+   * @param isdir if the path is a directory
+   * @param block_replication the file's replication factor
+   * @param blocksize a file's block size
+   * @param modification_time a file's modification time
+   * @param access_time a file's access time
+   * @param permission a file's permission
+   * @param owner a file's owner
+   * @param group a file's group
+   * @param symlink symlink if the path is a symbolic link
+   * @param path the path's qualified name
+   * @param attr Attribute flags (See {@link FileStatus.AttrFlags}).
+   * @param locations a file's block locations
+   */
+  public LocatedFileStatus(long length, boolean isdir, int block_replication,
+      long blocksize, long modification_time, long access_time,
+      FsPermission permission, String owner, String group, Path symlink,
+      Path path, Set<AttrFlags> attr, BlockLocation[] locations) {
+    super(length, isdir, block_replication, blocksize, modification_time,
+        access_time, permission, owner, group, symlink, path, attr);
     this.locations = locations;
   }
 
@@ -135,7 +162,16 @@ public class LocatedFileStatus extends FileStatus {
   public BlockLocation[] getBlockLocations() {
     return locations;
   }
-  
+
+  /**
+   * Hook for subclasses to lazily set block locations. The {@link #locations}
+   * field should be null before this is called.
+   * @param locations Block locations for this instance.
+   */
+  protected void setBlockLocations(BlockLocation[] locations) {
+    this.locations = locations;
+  }
+
   /**
    * Compare this FileStatus to another FileStatus
    * @param   o the FileStatus to be compared.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e560f3b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/PBHelper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/PBHelper.java
index 23caf2e..3b6724a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/PBHelper.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/protocolPB/PBHelper.java
@@ -96,12 +96,11 @@ public final class PBHelper {
     int flags = proto.getFlags();
     FileStatus fileStatus = new FileStatus(length, isdir, blockReplication,
         blocksize, mtime, atime, permission, owner, group, symlink, path,
-        (flags & FileStatusProto.Flags.HAS_ACL_VALUE) != 0,
-        (flags & FileStatusProto.Flags.HAS_CRYPT_VALUE) != 0,
-        (flags & FileStatusProto.Flags.HAS_EC_VALUE) != 0);
-
-    fileStatus.setSnapShotEnabledFlag((flags & FileStatusProto.Flags
-        .SNAPSHOT_ENABLED_VALUE) != 0);
+        FileStatus.attributes(
+          (flags & FileStatusProto.Flags.HAS_ACL_VALUE) != 0,
+          (flags & FileStatusProto.Flags.HAS_CRYPT_VALUE) != 0,
+          (flags & FileStatusProto.Flags.HAS_EC_VALUE) != 0,
+          (flags & FileStatusProto.Flags.SNAPSHOT_ENABLED_VALUE) != 0));
     return fileStatus;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e560f3b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
index 9d6ab9a..22ef722 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
@@ -60,4 +60,34 @@
     <Field name="cachingStrategy" />
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>
+
+  <!-- BlockLocations are user-facing, but LocatedBlocks are not. -->
+  <Match>
+    <Class name="org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus" />
+    <Field name="hdfsloc" />
+    <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED" />
+  </Match>
+
+  <!-- Hdfs*FileStatus are internal types. This "internal" state is not sensitive. -->
+  <Match>
+    <Class name="org.apache.hadoop.hdfs.protocol.HdfsNamedFileStatus" />
+    <Method name="getLocalNameInBytes" />
+    <Bug pattern="EI_EXPOSE_REP" />
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.hdfs.protocol.HdfsNamedFileStatus" />
+    <Method name="getSymlinkInBytes" />
+    <Bug pattern="EI_EXPOSE_REP" />
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus" />
+    <Method name="getLocalNameInBytes" />
+    <Bug pattern="EI_EXPOSE_REP" />
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus" />
+    <Method name="getSymlinkInBytes" />
+    <Bug pattern="EI_EXPOSE_REP" />
+  </Match>
+
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e560f3b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
index 0499f2e..264e3f4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
@@ -18,271 +18,40 @@
 package org.apache.hadoop.hdfs.protocol;
 
 import java.io.IOException;
+import java.io.ObjectInputValidation;
+import java.io.Serializable;
 import java.net.URI;
 import java.util.Arrays;
 import java.util.EnumSet;
+import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileStatus.AttrFlags;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.io.Writable;
 
-/** Interface that represents the over the wire information for a file.
+/**
+ * HDFS metadata for an entity in the filesystem.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class HdfsFileStatus extends FileStatus {
-
-  private static final long serialVersionUID = 0x126eb82a;
-
-  // local name of the inode that's encoded in java UTF8
-  private byte[] uPath;
-  private byte[] uSymlink; // symlink target encoded in java UTF8/null
-  private final long fileId;
-  private final FileEncryptionInfo feInfo;
-  private final ErasureCodingPolicy ecPolicy;
+public interface HdfsFileStatus
+    extends Writable, Comparable<Object>, Serializable, ObjectInputValidation {
 
-  // Used by dir, not including dot and dotdot. Always zero for a regular file.
-  private final int childrenNum;
-  private final byte storagePolicy;
+  byte[] EMPTY_NAME = new byte[0];
 
-  public static final byte[] EMPTY_NAME = new byte[0];
-
-  /**
-   * Set of features potentially active on an instance.
-   */
-  public enum Flags {
+  /** Set of features potentially active on an instance. */
+  enum Flags {
     HAS_ACL,
     HAS_CRYPT,
     HAS_EC,
     SNAPSHOT_ENABLED
   }
-  private final EnumSet<Flags> flags;
-
-  /**
-   * Constructor.
-   * @param length the number of bytes the file has
-   * @param isdir if the path is a directory
-   * @param replication the replication factor
-   * @param blocksize the block size
-   * @param mtime modification time
-   * @param atime access time
-   * @param permission permission
-   * @param owner the owner of the path
-   * @param group the group of the path
-   * @param symlink symlink target encoded in java UTF8 or null
-   * @param path the local name in java UTF8 encoding the same as that in-memory
-   * @param fileId the file id
-   * @param childrenNum the number of children. Used by directory.
-   * @param feInfo the file's encryption info
-   * @param storagePolicy ID which specifies storage policy
-   * @param ecPolicy the erasure coding policy
-   */
-  protected HdfsFileStatus(long length, boolean isdir, int replication,
-                         long blocksize, long mtime, long atime,
-                         FsPermission permission, EnumSet<Flags> flags,
-                         String owner, String group,
-                         byte[] symlink, byte[] path, long fileId,
-                         int childrenNum, FileEncryptionInfo feInfo,
-                         byte storagePolicy, ErasureCodingPolicy ecPolicy) {
-    super(length, isdir, replication, blocksize, mtime,
-        atime, convert(isdir, symlink != null, permission, flags),
-        owner, group, null, null,
-        flags.contains(Flags.HAS_ACL), flags.contains(Flags.HAS_CRYPT),
-        flags.contains(Flags.HAS_EC));
-    this.flags = flags;
-    this.uSymlink = symlink;
-    this.uPath = path;
-    this.fileId = fileId;
-    this.childrenNum = childrenNum;
-    this.feInfo = feInfo;
-    this.storagePolicy = storagePolicy;
-    this.ecPolicy = ecPolicy;
-  }
-
-  /**
-   * Set redundant flags for compatibility with existing applications.
-   */
-  protected static FsPermission convert(boolean isdir, boolean symlink,
-      FsPermission p, EnumSet<Flags> f) {
-    if (p instanceof FsPermissionExtension) {
-      // verify flags are set consistently
-      assert p.getAclBit() == f.contains(HdfsFileStatus.Flags.HAS_ACL);
-      assert p.getEncryptedBit() == f.contains(HdfsFileStatus.Flags.HAS_CRYPT);
-      assert p.getErasureCodedBit() == f.contains(HdfsFileStatus.Flags.HAS_EC);
-      return p;
-    }
-    if (null == p) {
-      if (isdir) {
-        p = FsPermission.getDirDefault();
-      } else if (symlink) {
-        p = FsPermission.getDefault();
-      } else {
-        p = FsPermission.getFileDefault();
-      }
-    }
-    return new FsPermissionExtension(p, f.contains(Flags.HAS_ACL),
-        f.contains(Flags.HAS_CRYPT), f.contains(Flags.HAS_EC));
-  }
-
-  @Override
-  public boolean isSymlink() {
-    return uSymlink != null;
-  }
-
-  @Override
-  public boolean hasAcl() {
-    return flags.contains(Flags.HAS_ACL);
-  }
-
-  @Override
-  public boolean isEncrypted() {
-    return flags.contains(Flags.HAS_CRYPT);
-  }
-
-  @Override
-  public boolean isErasureCoded() {
-    return flags.contains(Flags.HAS_EC);
-  }
-
-  /**
-   * Check if the local name is empty.
-   * @return true if the name is empty
-   */
-  public final boolean isEmptyLocalName() {
-    return uPath.length == 0;
-  }
-
-  /**
-   * Get the string representation of the local name.
-   * @return the local name in string
-   */
-  public final String getLocalName() {
-    return DFSUtilClient.bytes2String(uPath);
-  }
-
-  /**
-   * Get the Java UTF8 representation of the local name.
-   * @return the local name in java UTF8
-   */
-  public final byte[] getLocalNameInBytes() {
-    return uPath;
-  }
-
-  /**
-   * Get the string representation of the full path name.
-   * @param parent the parent path
-   * @return the full path in string
-   */
-  public final String getFullName(final String parent) {
-    if (isEmptyLocalName()) {
-      return parent;
-    }
-
-    StringBuilder fullName = new StringBuilder(parent);
-    if (!parent.endsWith(Path.SEPARATOR)) {
-      fullName.append(Path.SEPARATOR);
-    }
-    fullName.append(getLocalName());
-    return fullName.toString();
-  }
-
-  /**
-   * Get the full path.
-   * @param parent the parent path
-   * @return the full path
-   */
-  public final Path getFullPath(final Path parent) {
-    if (isEmptyLocalName()) {
-      return parent;
-    }
-
-    return new Path(parent, getLocalName());
-  }
-
-  @Override
-  public Path getSymlink() throws IOException {
-    if (isSymlink()) {
-      return new Path(DFSUtilClient.bytes2String(uSymlink));
-    }
-    throw new IOException("Path " + getPath() + " is not a symbolic link");
-  }
-
-  @Override
-  public void setSymlink(Path sym) {
-    uSymlink = DFSUtilClient.string2Bytes(sym.toString());
-  }
-
-  /**
-   * Opaque referant for the symlink, to be resolved at the client.
-   */
-  public final byte[] getSymlinkInBytes() {
-    return uSymlink;
-  }
-
-  public final long getFileId() {
-    return fileId;
-  }
-
-  public final FileEncryptionInfo getFileEncryptionInfo() {
-    return feInfo;
-  }
-
-  /**
-   * Get the erasure coding policy if it's set.
-   * @return the erasure coding policy
-   */
-  public ErasureCodingPolicy getErasureCodingPolicy() {
-    return ecPolicy;
-  }
-
-  public final int getChildrenNum() {
-    return childrenNum;
-  }
-
-  /** @return the storage policy id */
-  public final byte getStoragePolicy() {
-    return storagePolicy;
-  }
-
-  /**
-   * Check if directory is Snapshot enabled or not.
-   *
-   * @return true if directory is snapshot enabled
-   */
-  public boolean isSnapshotEnabled() {
-    return flags.contains(Flags.SNAPSHOT_ENABLED);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    // satisfy findbugs
-    return super.equals(o);
-  }
-
-  @Override
-  public int hashCode() {
-    // satisfy findbugs
-    return super.hashCode();
-  }
-
-  /**
-   * Resolve the short name of the Path given the URI, parent provided. This
-   * FileStatus reference will not contain a valid Path until it is resolved
-   * by this method.
-   * @param defaultUri FileSystem to fully qualify HDFS path.
-   * @param parent Parent path of this element.
-   * @return Reference to this instance.
-   */
-  public final FileStatus makeQualified(URI defaultUri, Path parent) {
-    // fully-qualify path
-    setPath(getFullPath(parent).makeQualified(defaultUri, null));
-    return this; // API compatibility
-
-  }
 
   /**
    * Builder class for HdfsFileStatus instances. Note default values for
@@ -290,7 +59,7 @@ public class HdfsFileStatus extends FileStatus {
    */
   @InterfaceAudience.Private
   @InterfaceStability.Unstable
-  public static class Builder {
+  class Builder {
     // Changing default values will affect cases where values are not
     // specified. Be careful!
     private long length                    = 0L;
@@ -311,6 +80,7 @@ public class HdfsFileStatus extends FileStatus {
     private byte storagePolicy             =
         HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
     private ErasureCodingPolicy ecPolicy   = null;
+    private LocatedBlocks locations        = null;
 
     /**
      * Set the length of the entity (default = 0).
@@ -490,13 +260,279 @@ public class HdfsFileStatus extends FileStatus {
     }
 
     /**
+     * Set the block locations for this entity (default = null).
+     * @param locations HDFS locations
+     *       (see {@link HdfsLocatedFileStatus#makeQualifiedLocated(URI, Path)})
+     * @return This Builder instance
+     */
+    public Builder locations(LocatedBlocks locations) {
+      this.locations = locations;
+      return this;
+    }
+
+    /**
      * @return An {@link HdfsFileStatus} instance from these parameters.
      */
     public HdfsFileStatus build() {
-      return new HdfsFileStatus(length, isdir, replication, blocksize,
-          mtime, atime, permission, flags, owner, group, symlink, path, fileId,
-          childrenNum, feInfo, storagePolicy, ecPolicy);
+      if (null == locations && !isdir && null == symlink) {
+        return new HdfsNamedFileStatus(length, isdir, replication, blocksize,
+            mtime, atime, permission, flags, owner, group, symlink, path,
+            fileId, childrenNum, feInfo, storagePolicy, ecPolicy);
+      }
+      return new HdfsLocatedFileStatus(length, isdir, replication, blocksize,
+          mtime, atime, permission, flags, owner, group, symlink, path,
+          fileId, childrenNum, feInfo, storagePolicy, ecPolicy, locations);
+    }
+
+  }
+
+  ///////////////////
+  // HDFS-specific //
+  ///////////////////
+
+  /**
+   * Inode ID for this entity, if a file.
+   * @return inode ID.
+   */
+  long getFileId();
+
+  /**
+   * Get metadata for encryption, if present.
+   * @return the {@link FileEncryptionInfo} for this stream, or null if not
+   *         encrypted.
+   */
+  FileEncryptionInfo getFileEncryptionInfo();
+
+  /**
+   * Check if the local name is empty.
+   * @return true if the name is empty
+   */
+  default boolean isEmptyLocalName() {
+    return getLocalNameInBytes().length == 0;
+  }
+
+  /**
+   * Get the string representation of the local name.
+   * @return the local name in string
+   */
+  default String getLocalName() {
+    return DFSUtilClient.bytes2String(getLocalNameInBytes());
+  }
+
+  /**
+   * Get the Java UTF8 representation of the local name.
+   * @return the local name in java UTF8
+   */
+  byte[] getLocalNameInBytes();
+
+  /**
+   * Get the string representation of the full path name.
+   * @param parent the parent path
+   * @return the full path in string
+   */
+  default String getFullName(String parent) {
+    if (isEmptyLocalName()) {
+      return parent;
+    }
+
+    StringBuilder fullName = new StringBuilder(parent);
+    if (!parent.endsWith(Path.SEPARATOR)) {
+      fullName.append(Path.SEPARATOR);
+    }
+    fullName.append(getLocalName());
+    return fullName.toString();
+  }
+
+  /**
+   * Get the full path.
+   * @param parent the parent path
+   * @return the full path
+   */
+  default Path getFullPath(Path parent) {
+    if (isEmptyLocalName()) {
+      return parent;
+    }
+
+    return new Path(parent, getLocalName());
+  }
+
+  /**
+   * Opaque referant for the symlink, to be resolved at the client.
+   */
+  byte[] getSymlinkInBytes();
+
+  /**
+   * @return number of children for this inode.
+   */
+  int getChildrenNum();
+
+  /**
+   * Get the erasure coding policy if it's set.
+   * @return the erasure coding policy
+   */
+  ErasureCodingPolicy getErasureCodingPolicy();
+
+  /** @return the storage policy id */
+  byte getStoragePolicy();
+
+  /**
+   * Resolve the short name of the Path given the URI, parent provided. This
+   * FileStatus reference will not contain a valid Path until it is resolved
+   * by this method.
+   * @param defaultUri FileSystem to fully qualify HDFS path.
+   * @param parent Parent path of this element.
+   * @return Reference to this instance.
+   */
+  default FileStatus makeQualified(URI defaultUri, Path parent) {
+    // fully-qualify path
+    setPath(getFullPath(parent).makeQualified(defaultUri, null));
+    return (FileStatus) this; // API compatibility
+  }
+
+  ////////////////////////////
+  // FileStatus "overrides" //
+  ////////////////////////////
+
+  /**
+   * See {@link FileStatus#getPath()}.
+   */
+  Path getPath();
+  /**
+   * See {@link FileStatus#setPath(Path)}.
+   */
+  void setPath(Path p);
+  /**
+   * See {@link FileStatus#getLen()}.
+   */
+  long getLen();
+  /**
+   * See {@link FileStatus#isFile()}.
+   */
+  boolean isFile();
+  /**
+   * See {@link FileStatus#isDirectory()}.
+   */
+  boolean isDirectory();
+  /**
+   * See {@link FileStatus#isDir()}.
+   */
+  boolean isDir();
+  /**
+   * See {@link FileStatus#isSymlink()}.
+   */
+  boolean isSymlink();
+  /**
+   * See {@link FileStatus#getBlockSize()}.
+   */
+  long getBlockSize();
+  /**
+   * See {@link FileStatus#getReplication()}.
+   */
+  short getReplication();
+  /**
+   * See {@link FileStatus#getModificationTime()}.
+   */
+  long getModificationTime();
+  /**
+   * See {@link FileStatus#getAccessTime()}.
+   */
+  long getAccessTime();
+  /**
+   * See {@link FileStatus#getPermission()}.
+   */
+  FsPermission getPermission();
+  /**
+   * See {@link FileStatus#setPermission(FsPermission)}.
+   */
+  void setPermission(FsPermission permission);
+  /**
+   * See {@link FileStatus#getOwner()}.
+   */
+  String getOwner();
+  /**
+   * See {@link FileStatus#setOwner(String)}.
+   */
+  void setOwner(String owner);
+  /**
+   * See {@link FileStatus#getGroup()}.
+   */
+  String getGroup();
+  /**
+   * See {@link FileStatus#setGroup(String)}.
+   */
+  void setGroup(String group);
+  /**
+   * See {@link FileStatus#hasAcl()}.
+   */
+  boolean hasAcl();
+  /**
+   * See {@link FileStatus#isEncrypted()}.
+   */
+  boolean isEncrypted();
+  /**
+   * See {@link FileStatus#isErasureCoded()}.
+   */
+  boolean isErasureCoded();
+  /**
+   * See {@link FileStatus#isSnapshotEnabled()}.
+   */
+  boolean isSnapshotEnabled();
+  /**
+   * See {@link FileStatus#getSymlink()}.
+   */
+  Path getSymlink() throws IOException;
+  /**
+   * See {@link FileStatus#setSymlink(Path sym)}.
+   */
+  void setSymlink(Path sym);
+  /**
+   * See {@link FileStatus#compareTo(FileStatus)}.
+   */
+  int compareTo(FileStatus stat);
+
+  /**
+   * Set redundant flags for compatibility with existing applications.
+   */
+  static FsPermission convert(boolean isdir, boolean symlink,
+                              FsPermission p, Set<Flags> f) {
+    if (p instanceof FsPermissionExtension) {
+      // verify flags are set consistently
+      assert p.getAclBit() == f.contains(HdfsFileStatus.Flags.HAS_ACL);
+      assert p.getEncryptedBit() == f.contains(HdfsFileStatus.Flags.HAS_CRYPT);
+      assert p.getErasureCodedBit() == f.contains(HdfsFileStatus.Flags.HAS_EC);
+      return p;
+    }
+    if (null == p) {
+      if (isdir) {
+        p = FsPermission.getDirDefault();
+      } else if (symlink) {
+        p = FsPermission.getDefault();
+      } else {
+        p = FsPermission.getFileDefault();
+      }
+    }
+    return new FsPermissionExtension(p, f.contains(Flags.HAS_ACL),
+        f.contains(Flags.HAS_CRYPT), f.contains(Flags.HAS_EC));
+  }
+
+  static Set<AttrFlags> convert(Set<Flags> flags) {
+    if (flags.isEmpty()) {
+      return FileStatus.NONE;
+    }
+    EnumSet<AttrFlags> attr = EnumSet.noneOf(AttrFlags.class);
+    if (flags.contains(Flags.HAS_ACL)) {
+      attr.add(AttrFlags.HAS_ACL);
+    }
+    if (flags.contains(Flags.HAS_EC)) {
+      attr.add(AttrFlags.HAS_EC);
+    }
+    if (flags.contains(Flags.HAS_CRYPT)) {
+      attr.add(AttrFlags.HAS_CRYPT);
+    }
+    if (flags.contains(Flags.SNAPSHOT_ENABLED)) {
+      attr.add(AttrFlags.SNAPSHOT_ENABLED);
     }
+    return attr;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e560f3b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
index 193aae2..1490e4e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.protocol;
 
+import java.io.IOException;
 import java.net.URI;
 import java.util.EnumSet;
 
@@ -29,71 +30,150 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 
 /**
- * Interface that represents the over the wire information
- * including block locations for a file.
+ * HDFS metadata for an entity in the filesystem with locations. Note that
+ * symlinks and directories are returned as {@link HdfsLocatedFileStatus} for
+ * backwards compatibility.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class HdfsLocatedFileStatus extends HdfsFileStatus {
+public class HdfsLocatedFileStatus
+    extends LocatedFileStatus implements HdfsFileStatus {
+  private static final long serialVersionUID = 0x126eb82a;
 
-  private static final long serialVersionUID = 0x23c73328;
+  // local name of the inode that's encoded in java UTF8
+  private byte[] uPath;
+  private byte[] uSymlink; // symlink target encoded in java UTF8/null
+  private final long fileId;
+  private final FileEncryptionInfo feInfo;
+  private final ErasureCodingPolicy ecPolicy;
 
-  /**
-   * Left transient, because {@link #makeQualifiedLocated(URI,Path)}
-   * is the user-facing type.
-   */
-  private transient LocatedBlocks locations;
+  // Used by dir, not including dot and dotdot. Always zero for a regular file.
+  private final int childrenNum;
+  private final byte storagePolicy;
+
+  // BlockLocations[] is the user-facing type
+  private transient LocatedBlocks hdfsloc;
 
   /**
-   * Constructor
-   *
-   * @param length size
-   * @param isdir if this is directory
-   * @param block_replication the file's replication factor
-   * @param blocksize the file's block size
-   * @param modification_time most recent modification time
-   * @param access_time most recent access time
+   * Constructor.
+   * @param length the number of bytes the file has
+   * @param isdir if the path is a directory
+   * @param replication the replication factor
+   * @param blocksize the block size
+   * @param mtime modification time
+   * @param atime access time
    * @param permission permission
-   * @param owner owner
-   * @param group group
-   * @param symlink symbolic link
-   * @param path local path name in java UTF8 format
+   * @param owner the owner of the path
+   * @param group the group of the path
+   * @param symlink symlink target encoded in java UTF8 or null
+   * @param path the local name in java UTF8 encoding the same as that in-memory
    * @param fileId the file id
-   * @param locations block locations
-   * @param feInfo file encryption info
+   * @param childrenNum the number of children. Used by directory.
+   * @param feInfo the file's encryption info
+   * @param storagePolicy ID which specifies storage policy
+   * @param ecPolicy the erasure coding policy
+   * @param hdfsloc block locations
    */
-  public HdfsLocatedFileStatus(long length, boolean isdir,
-      int block_replication, long blocksize, long modification_time,
-      long access_time, FsPermission permission, EnumSet<Flags> flags,
-      String owner, String group, byte[] symlink, byte[] path, long fileId,
-      LocatedBlocks locations, int childrenNum, FileEncryptionInfo feInfo,
-      byte storagePolicy, ErasureCodingPolicy ecPolicy) {
-    super(length, isdir, block_replication, blocksize, modification_time,
-        access_time, permission, flags, owner, group, symlink, path, fileId,
-        childrenNum, feInfo, storagePolicy, ecPolicy);
-    this.locations = locations;
+  HdfsLocatedFileStatus(long length, boolean isdir, int replication,
+                        long blocksize, long mtime, long atime,
+                        FsPermission permission, EnumSet<Flags> flags,
+                        String owner, String group,
+                        byte[] symlink, byte[] path, long fileId,
+                        int childrenNum, FileEncryptionInfo feInfo,
+                        byte storagePolicy, ErasureCodingPolicy ecPolicy,
+                        LocatedBlocks hdfsloc) {
+    super(length, isdir, replication, blocksize, mtime, atime,
+        HdfsFileStatus.convert(isdir, symlink != null, permission, flags),
+        owner, group, null, null, HdfsFileStatus.convert(flags),
+        null);
+    this.uSymlink = symlink;
+    this.uPath = path;
+    this.fileId = fileId;
+    this.childrenNum = childrenNum;
+    this.feInfo = feInfo;
+    this.storagePolicy = storagePolicy;
+    this.ecPolicy = ecPolicy;
+    this.hdfsloc = hdfsloc;
   }
 
-  public LocatedBlocks getBlockLocations() {
-    return locations;
+  @Override // visibility
+  public void setOwner(String owner) {
+    super.setOwner(owner);
+  }
+
+  @Override // visibility
+  public void setGroup(String group) {
+    super.setOwner(group);
+  }
+
+  @Override
+  public boolean isSymlink() {
+    return uSymlink != null;
+  }
+
+  @Override
+  public Path getSymlink() throws IOException {
+    if (isSymlink()) {
+      return new Path(DFSUtilClient.bytes2String(getSymlinkInBytes()));
+    }
+    throw new IOException("Path " + getPath() + " is not a symbolic link");
+  }
+
+  @Override // visibility
+  public void setPermission(FsPermission permission) {
+    super.setPermission(permission);
   }
 
   /**
-   * This function is used to transform the underlying HDFS LocatedBlocks to
-   * BlockLocations.
-   *
-   * The returned BlockLocation will have different formats for replicated
-   * and erasure coded file.
-   * Please refer to
-   * {@link org.apache.hadoop.fs.FileSystem#getFileBlockLocations
-   * (FileStatus, long, long)}
-   * for examples.
+   * Get the Java UTF8 representation of the local name.
+   * @return the local name in java UTF8
    */
-  public final LocatedFileStatus makeQualifiedLocated(URI defaultUri,
-      Path path) {
-    makeQualified(defaultUri, path);
-    return new LocatedFileStatus(this,
-        DFSUtilClient.locatedBlocks2Locations(getBlockLocations()));
+  @Override
+  public byte[] getLocalNameInBytes() {
+    return uPath;
+  }
+
+  @Override
+  public void setSymlink(Path sym) {
+    uSymlink = DFSUtilClient.string2Bytes(sym.toString());
+  }
+
+  /**
+   * Opaque referant for the symlink, to be resolved at the client.
+   */
+  @Override
+  public byte[] getSymlinkInBytes() {
+    return uSymlink;
+  }
+
+  @Override
+  public long getFileId() {
+    return fileId;
+  }
+
+  @Override
+  public FileEncryptionInfo getFileEncryptionInfo() {
+    return feInfo;
+  }
+
+  /**
+   * Get the erasure coding policy if it's set.
+   * @return the erasure coding policy
+   */
+  @Override
+  public ErasureCodingPolicy getErasureCodingPolicy() {
+    return ecPolicy;
+  }
+
+  @Override
+  public int getChildrenNum() {
+    return childrenNum;
+  }
+
+  /** @return the storage policy id */
+  @Override
+  public byte getStoragePolicy() {
+    return storagePolicy;
   }
 
   @Override
@@ -107,4 +187,34 @@ public class HdfsLocatedFileStatus extends HdfsFileStatus {
     // satisfy findbugs
     return super.hashCode();
   }
+
+  /**
+   * Get block locations for this entity, in HDFS format.
+   * See {@link #makeQualifiedLocated(URI, Path)}.
+   * See {@link DFSUtilClient#locatedBlocks2Locations(LocatedBlocks)}.
+   * @return block locations
+   */
+  public LocatedBlocks getLocatedBlocks() {
+    return hdfsloc;
+  }
+
+  /**
+   * This function is used to transform the underlying HDFS LocatedBlocks to
+   * BlockLocations. This method must be invoked before
+   * {@link #getBlockLocations()}.
+   *
+   * The returned BlockLocation will have different formats for replicated
+   * and erasure coded file.
+   * Please refer to
+   * {@link org.apache.hadoop.fs.FileSystem#getFileBlockLocations
+   * (FileStatus, long, long)}
+   * for examples.
+   */
+  public LocatedFileStatus makeQualifiedLocated(URI defaultUri, Path path) {
+    makeQualified(defaultUri, path);
+    setBlockLocations(
+        DFSUtilClient.locatedBlocks2Locations(getLocatedBlocks()));
+    return this;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e560f3b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsNamedFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsNamedFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsNamedFileStatus.java
new file mode 100644
index 0000000..311f9d0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsNamedFileStatus.java
@@ -0,0 +1,180 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+
+import java.io.IOException;
+import java.util.Set;
+
+/**
+ * HDFS metadata for an entity in the filesystem without locations. Note that
+ * symlinks and directories are returned as {@link HdfsLocatedFileStatus} for
+ * backwards compatibility.
+ */
+public class HdfsNamedFileStatus extends FileStatus implements HdfsFileStatus {
+
+  // local name of the inode that's encoded in java UTF8
+  private byte[] uPath;
+  private byte[] uSymlink; // symlink target encoded in java UTF8/null
+  private final long fileId;
+  private final FileEncryptionInfo feInfo;
+  private final ErasureCodingPolicy ecPolicy;
+
+  // Used by dir, not including dot and dotdot. Always zero for a regular file.
+  private final int childrenNum;
+  private final byte storagePolicy;
+
+  /**
+   * Constructor.
+   * @param length the number of bytes the file has
+   * @param isdir if the path is a directory
+   * @param replication the replication factor
+   * @param blocksize the block size
+   * @param mtime modification time
+   * @param atime access time
+   * @param permission permission
+   * @param owner the owner of the path
+   * @param group the group of the path
+   * @param symlink symlink target encoded in java UTF8 or null
+   * @param path the local name in java UTF8 encoding the same as that in-memory
+   * @param fileId the file id
+   * @param childrenNum the number of children. Used by directory.
+   * @param feInfo the file's encryption info
+   * @param storagePolicy ID which specifies storage policy
+   * @param ecPolicy the erasure coding policy
+   */
+  HdfsNamedFileStatus(long length, boolean isdir, int replication,
+                      long blocksize, long mtime, long atime,
+                      FsPermission permission, Set<Flags> flags,
+                      String owner, String group,
+                      byte[] symlink, byte[] path, long fileId,
+                      int childrenNum, FileEncryptionInfo feInfo,
+                      byte storagePolicy, ErasureCodingPolicy ecPolicy) {
+    super(length, isdir, replication, blocksize, mtime, atime,
+        HdfsFileStatus.convert(isdir, symlink != null, permission, flags),
+        owner, group, null, null,
+        HdfsFileStatus.convert(flags));
+    this.uSymlink = symlink;
+    this.uPath = path;
+    this.fileId = fileId;
+    this.childrenNum = childrenNum;
+    this.feInfo = feInfo;
+    this.storagePolicy = storagePolicy;
+    this.ecPolicy = ecPolicy;
+  }
+
+  @Override
+  public void setOwner(String owner) {
+    super.setOwner(owner);
+  }
+
+  @Override
+  public void setGroup(String group) {
+    super.setOwner(group);
+  }
+
+  @Override
+  public boolean isSymlink() {
+    return uSymlink != null;
+  }
+
+  @Override
+  public Path getSymlink() throws IOException {
+    if (isSymlink()) {
+      return new Path(DFSUtilClient.bytes2String(getSymlinkInBytes()));
+    }
+    throw new IOException("Path " + getPath() + " is not a symbolic link");
+  }
+
+  @Override
+  public void setPermission(FsPermission permission) {
+    super.setPermission(permission);
+  }
+
+  /**
+   * Get the Java UTF8 representation of the local name.
+   *
+   * @return the local name in java UTF8
+   */
+  @Override
+  public byte[] getLocalNameInBytes() {
+    return uPath;
+  }
+
+  @Override
+  public void setSymlink(Path sym) {
+    uSymlink = DFSUtilClient.string2Bytes(sym.toString());
+  }
+
+  /**
+   * Opaque referant for the symlink, to be resolved at the client.
+   */
+  @Override
+  public byte[] getSymlinkInBytes() {
+    return uSymlink;
+  }
+
+  @Override
+  public long getFileId() {
+    return fileId;
+  }
+
+  @Override
+  public FileEncryptionInfo getFileEncryptionInfo() {
+    return feInfo;
+  }
+
+  /**
+   * Get the erasure coding policy if it's set.
+   *
+   * @return the erasure coding policy
+   */
+  @Override
+  public ErasureCodingPolicy getErasureCodingPolicy() {
+    return ecPolicy;
+  }
+
+  @Override
+  public int getChildrenNum() {
+    return childrenNum;
+  }
+
+  /** @return the storage policy id */
+  @Override
+  public byte getStoragePolicy() {
+    return storagePolicy;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    // satisfy findbugs
+    return super.equals(o);
+  }
+
+  @Override
+  public int hashCode() {
+    // satisfy findbugs
+    return super.hashCode();
+  }
+
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e560f3b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index ae3921c..d3b7f6d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -1585,23 +1585,36 @@ public class PBHelperClient {
     EnumSet<HdfsFileStatus.Flags> flags = fs.hasFlags()
         ? convertFlags(fs.getFlags())
         : convertFlags(fs.getPermission());
-    return new HdfsLocatedFileStatus(
-        fs.getLength(), fs.getFileType().equals(FileType.IS_DIR),
-        fs.getBlockReplication(), fs.getBlocksize(),
-        fs.getModificationTime(), fs.getAccessTime(),
-        convert(fs.getPermission()),
-        flags,
-        fs.getOwner(), fs.getGroup(),
-        fs.getFileType().equals(FileType.IS_SYMLINK) ?
-            fs.getSymlink().toByteArray() : null,
-        fs.getPath().toByteArray(),
-        fs.hasFileId()? fs.getFileId(): HdfsConstants.GRANDFATHER_INODE_ID,
-        fs.hasLocations() ? convert(fs.getLocations()) : null,
-        fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
-        fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
-        fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
-            : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
-        fs.hasEcPolicy() ? convertErasureCodingPolicy(fs.getEcPolicy()) : null);
+    return new HdfsFileStatus.Builder()
+        .length(fs.getLength())
+        .isdir(fs.getFileType().equals(FileType.IS_DIR))
+        .replication(fs.getBlockReplication())
+        .blocksize(fs.getBlocksize())
+        .mtime(fs.getModificationTime())
+        .atime(fs.getAccessTime())
+        .perm(convert(fs.getPermission()))
+        .flags(flags)
+        .owner(fs.getOwner())
+        .group(fs.getGroup())
+        .symlink(FileType.IS_SYMLINK.equals(fs.getFileType())
+            ? fs.getSymlink().toByteArray()
+            : null)
+        .path(fs.getPath().toByteArray())
+        .fileId(fs.hasFileId()
+            ? fs.getFileId()
+            : HdfsConstants.GRANDFATHER_INODE_ID)
+        .locations(fs.hasLocations() ? convert(fs.getLocations()) : null)
+        .children(fs.hasChildrenNum() ? fs.getChildrenNum() : -1)
+        .feInfo(fs.hasFileEncryptionInfo()
+            ? convert(fs.getFileEncryptionInfo())
+            : null)
+        .storagePolicy(fs.hasStoragePolicy()
+            ? (byte) fs.getStoragePolicy()
+            : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED)
+        .ecPolicy(fs.hasEcPolicy()
+            ? convertErasureCodingPolicy(fs.getEcPolicy())
+            : null)
+        .build();
   }
 
   private static EnumSet<HdfsFileStatus.Flags> convertFlags(int flags) {
@@ -1864,10 +1877,10 @@ public class PBHelperClient {
     if (dl == null)
       return null;
     List<HdfsFileStatusProto> partList =  dl.getPartialListingList();
-    return new DirectoryListing(partList.isEmpty() ?
-        new HdfsLocatedFileStatus[0] :
-        convert(partList.toArray(new HdfsFileStatusProto[partList.size()])),
-        dl.getRemainingEntries());
+    return new DirectoryListing(partList.isEmpty()
+        ? new HdfsFileStatus[0]
+        : convert(partList.toArray(new HdfsFileStatusProto[partList.size()])),
+                  dl.getRemainingEntries());
   }
 
   public static HdfsFileStatus[] convert(HdfsFileStatusProto[] fs) {
@@ -2163,7 +2176,7 @@ public class PBHelperClient {
     }
     if (fs instanceof HdfsLocatedFileStatus) {
       final HdfsLocatedFileStatus lfs = (HdfsLocatedFileStatus) fs;
-      LocatedBlocks locations = lfs.getBlockLocations();
+      LocatedBlocks locations = lfs.getLocatedBlocks();
       if (locations != null) {
         builder.setLocations(convert(locations));
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e560f3b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestHdfsFileStatusMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestHdfsFileStatusMethods.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestHdfsFileStatusMethods.java
new file mode 100644
index 0000000..3cc4190
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestHdfsFileStatusMethods.java
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Set;
+import java.util.stream.Stream;
+import static java.util.stream.Collectors.joining;
+import static java.util.stream.Collectors.toSet;
+
+import org.apache.hadoop.fs.FileStatus;
+
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Unit test verifying that {@link HdfsFileStatus} is a superset of
+ * {@link FileStatus}.
+ */
+public class TestHdfsFileStatusMethods {
+
+  @Test
+  public void testInterfaceSuperset() {
+    Set<MethodSignature>  fsM = signatures(FileStatus.class);
+    Set<MethodSignature> hfsM = signatures(HdfsFileStatus.class);
+    hfsM.addAll(Stream.of(HdfsFileStatus.class.getInterfaces())
+        .flatMap(i -> Stream.of(i.getDeclaredMethods()))
+        .map(MethodSignature::new)
+        .collect(toSet()));
+    // HdfsFileStatus is not a concrete type
+    hfsM.addAll(signatures(Object.class));
+    assertTrue(fsM.removeAll(hfsM));
+    // verify that FileStatus is a subset of HdfsFileStatus
+    assertEquals(fsM.stream()
+            .map(MethodSignature::toString)
+            .collect(joining("\n")),
+        Collections.EMPTY_SET, fsM);
+  }
+
+  /** Map non-static, declared methods for this class to signatures. */
+  private static Set<MethodSignature> signatures(Class<?> c) {
+    return Stream.of(c.getDeclaredMethods())
+        .filter(m -> !Modifier.isStatic(m.getModifiers()))
+        .map(MethodSignature::new)
+        .collect(toSet());
+  }
+
+  private static class MethodSignature {
+    private final String name;
+    private final Type rval;
+    private final Type[] param;
+    MethodSignature(Method m) {
+      name = m.getName();
+      rval = m.getGenericReturnType();
+      param = m.getParameterTypes();
+    }
+    @Override
+    public int hashCode() {
+      return name.hashCode();
+    }
+    /**
+     * Methods are equal iff they have the same name, return type, and params
+     * (non-generic).
+     */
+    @Override
+    public boolean equals(Object o) {
+      if (!(o instanceof MethodSignature)) {
+        return false;
+      }
+      MethodSignature s = (MethodSignature) o;
+      return name.equals(s.name) &&
+          rval.equals(s.rval) &&
+          Arrays.equals(param, s.param);
+    }
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append(rval).append(" ").append(name).append("(")
+        .append(Stream.of(param)
+            .map(Type::toString).collect(joining(",")))
+        .append(")");
+      return sb.toString();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e560f3b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
index ffd64a1..54eaa78 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
@@ -1085,10 +1085,7 @@ public class HttpFSFileSystem extends FileSystem
           new FsPermissionExtension(permission, aBit, eBit, ecBit);
       FileStatus fileStatus = new FileStatus(len, FILE_TYPE.DIRECTORY == type,
           replication, blockSize, mTime, aTime, deprecatedPerm, owner, group,
-          null, path, aBit, eBit, ecBit);
-      if (seBit) {
-        fileStatus.setSnapShotEnabledFlag(seBit);
-      }
+          null, path, FileStatus.attributes(aBit, eBit, ecBit, seBit));
       return fileStatus;
     } else {
       return new FileStatus(len, FILE_TYPE.DIRECTORY == type,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e560f3b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
index 4b958b5..a99ea75 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
@@ -260,13 +260,6 @@
         <Method name="visitFile" />
         <Bug pattern="NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE" />
     </Match>
-    <!-- HdfsFileStatus is user-facing, but HdfsLocatedFileStatus is not.
-         Defensible compatibility choices over time create odd corners. -->
-    <Match>
-        <Class name="org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus" />
-        <Field name="locations" />
-        <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED" />
-    </Match>
     <Match>
        <Class name="org.apache.hadoop.hdfs.server.namenode.NNUpgradeUtil$1" />
        <Method name="visitFile" />

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e560f3b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
index b653f4f..8b89378 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
@@ -395,7 +395,7 @@ public class Mover {
           status.getReplication());
 
       final ErasureCodingPolicy ecPolicy = status.getErasureCodingPolicy();
-      final LocatedBlocks locatedBlocks = status.getBlockLocations();
+      final LocatedBlocks locatedBlocks = status.getLocatedBlocks();
       final boolean lastBlkComplete = locatedBlocks.isLastBlockComplete();
       List<LocatedBlock> lbs = locatedBlocks.getLocatedBlocks();
       for (int i = 0; i < lbs.size(); i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e560f3b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index b6e38a7..4da8a28 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -261,7 +261,7 @@ class FSDirStatAndListingOp {
             // This helps to prevent excessively large response payloads.
             // Approximate #locations with locatedBlockCount() * repl_factor
             LocatedBlocks blks =
-                ((HdfsLocatedFileStatus)listing[i]).getBlockLocations();
+                ((HdfsLocatedFileStatus)listing[i]).getLocatedBlocks();
             locationBudget -= (blks == null) ? 0 :
                blks.locatedBlockCount() * listing[i].getReplication();
         }
@@ -486,31 +486,26 @@ class FSDirStatAndListingOp {
       String owner, String group, byte[] symlink, byte[] path, long fileId,
       int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy,
       ErasureCodingPolicy ecPolicy, LocatedBlocks locations) {
-    if (locations == null) {
-      return new HdfsFileStatus.Builder()
-          .length(length)
-          .isdir(isdir)
-          .replication(replication)
-          .blocksize(blocksize)
-          .mtime(mtime)
-          .atime(atime)
-          .perm(permission)
-          .flags(flags)
-          .owner(owner)
-          .group(group)
-          .symlink(symlink)
-          .path(path)
-          .fileId(fileId)
-          .children(childrenNum)
-          .feInfo(feInfo)
-          .storagePolicy(storagePolicy)
-          .ecPolicy(ecPolicy)
-          .build();
-    } else {
-      return new HdfsLocatedFileStatus(length, isdir, replication, blocksize,
-          mtime, atime, permission, flags, owner, group, symlink, path,
-          fileId, locations, childrenNum, feInfo, storagePolicy, ecPolicy);
-    }
+    return new HdfsFileStatus.Builder()
+        .length(length)
+        .isdir(isdir)
+        .replication(replication)
+        .blocksize(blocksize)
+        .mtime(mtime)
+        .atime(atime)
+        .perm(permission)
+        .flags(flags)
+        .owner(owner)
+        .group(group)
+        .symlink(symlink)
+        .path(path)
+        .fileId(fileId)
+        .children(childrenNum)
+        .feInfo(feInfo)
+        .storagePolicy(storagePolicy)
+        .ecPolicy(ecPolicy)
+        .locations(locations)
+        .build();
   }
 
   private static ContentSummary getContentSummaryInt(FSDirectory fsd,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e560f3b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
index 3a8fb59..ae256a5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
@@ -1096,7 +1096,7 @@ public class TestBlockStoragePolicy {
                                   int replicaNum, StorageType... types) {
     List<StorageType> typeList = Lists.newArrayList();
     Collections.addAll(typeList, types);
-    LocatedBlocks lbs = status.getBlockLocations();
+    LocatedBlocks lbs = status.getLocatedBlocks();
     Assert.assertEquals(blockNum, lbs.getLocatedBlocks().size());
     for (LocatedBlock lb : lbs.getLocatedBlocks()) {
       Assert.assertEquals(replicaNum, lb.getStorageTypes().length);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e560f3b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusSerialization.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusSerialization.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusSerialization.java
index 42c9acf..62d1b9f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusSerialization.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusSerialization.java
@@ -110,7 +110,7 @@ public class TestFileStatusSerialization {
       dib.reset(dob.getData(), 0, dob.getLength());
       FileStatus fstat = new FileStatus();
       fstat.readFields(dib);
-      checkFields(stat, fstat);
+      checkFields((FileStatus) stat, fstat);
 
       // FsPermisisonExtension used for HdfsFileStatus, not FileStatus,
       // attribute flags should still be preserved
@@ -133,7 +133,7 @@ public class TestFileStatusSerialization {
     try (ObjectInputStream ois = new ObjectInputStream(bais)) {
       FileStatus deser = (FileStatus) ois.readObject();
       assertEquals(hs, deser);
-      checkFields(hs, deser);
+      checkFields((FileStatus) hs, deser);
     }
   }
 
@@ -168,8 +168,8 @@ public class TestFileStatusSerialization {
       byte[] dst = fsp.toByteArray();
       HdfsFileStatusProto hsp2 = HdfsFileStatusProto.parseFrom(dst);
       assertEquals(hsp, hsp2);
-      FileStatus hstat  = PBHelperClient.convert(hsp);
-      FileStatus hstat2 = PBHelperClient.convert(hsp2);
+      FileStatus hstat  = (FileStatus) PBHelperClient.convert(hsp);
+      FileStatus hstat2 = (FileStatus) PBHelperClient.convert(hsp2);
       checkFields(hstat, hstat2);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e560f3b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
index 764a0db..356ae3a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
@@ -320,7 +320,7 @@ public class TestStorageMover {
       }
       final List<StorageType> types = policy.chooseStorageTypes(
           status.getReplication());
-      for(LocatedBlock lb : fileStatus.getBlockLocations().getLocatedBlocks()) {
+      for(LocatedBlock lb : fileStatus.getLocatedBlocks().getLocatedBlocks()) {
         final Mover.StorageTypeDiff diff = new Mover.StorageTypeDiff(types,
             lb.getStorageTypes());
         Assert.assertTrue(fileStatus.getFullName(parent.toString())


---------------------------------------------------------------------
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: YARN-6669. Implemented Kerberos security for YARN service framework. (Contributed by Jian He)

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceUtils.java
index e18bcae..173001b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceUtils.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.service.conf.YarnServiceConstants;
@@ -36,6 +37,7 @@ import org.apache.hadoop.yarn.service.exceptions.SliderException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
 import java.io.BufferedOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
@@ -43,9 +45,7 @@ import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.FilenameFilter;
 import java.io.IOException;
-import java.net.ServerSocket;
-import java.net.URL;
-import java.net.URLDecoder;
+import java.net.*;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -57,6 +57,11 @@ import java.util.Map;
 import java.util.regex.Pattern;
 import java.util.zip.GZIPOutputStream;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic
+    .HADOOP_SECURITY_DNS_INTERFACE_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic
+    .HADOOP_SECURITY_DNS_NAMESERVER_KEY;
+
 /**
  * These are slider-specific Util methods
  */
@@ -542,4 +547,24 @@ public final class ServiceUtils {
   public static String createDescriptionTag(String description) {
     return "Description: " + description;
   }
+
+  // Copied from SecurityUtil because it is not public
+  public static String getLocalHostName(@Nullable Configuration conf)
+      throws UnknownHostException {
+    if (conf != null) {
+      String dnsInterface = conf.get(HADOOP_SECURITY_DNS_INTERFACE_KEY);
+      String nameServer = conf.get(HADOOP_SECURITY_DNS_NAMESERVER_KEY);
+
+      if (dnsInterface != null) {
+        return DNS.getDefaultHost(dnsInterface, nameServer, true);
+      } else if (nameServer != null) {
+        throw new IllegalArgumentException(HADOOP_SECURITY_DNS_NAMESERVER_KEY +
+            " requires " + HADOOP_SECURITY_DNS_INTERFACE_KEY + ". Check your" +
+            "configuration.");
+      }
+    }
+
+    // Fallback to querying the default hostname as we did before.
+    return InetAddress.getLocalHost().getCanonicalHostName();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo
new file mode 100644
index 0000000..14cdf68
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo
@@ -0,0 +1,14 @@
+#
+#   Licensed under the Apache License, Version 2.0 (the "License");
+#   you may not use this file except in compliance with the License.
+#   You may obtain a copy of the License at
+#
+#       http://www.apache.org/licenses/LICENSE-2.0
+#
+#   Unless required by applicable law or agreed to in writing, software
+#   distributed under the License is distributed on an "AS IS" BASIS,
+#   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#   See the License for the specific language governing permissions and
+#   limitations under the License.
+#
+org.apache.hadoop.yarn.service.ClientAMSecurityInfo

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/RegistryOperationsFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/RegistryOperationsFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/RegistryOperationsFactory.java
index 704b097..e74ca81 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/RegistryOperationsFactory.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/api/RegistryOperationsFactory.java
@@ -111,6 +111,27 @@ public final class RegistryOperationsFactory {
   }
 
   /**
+   * Create a kerberos registry service client
+   * @param conf configuration
+   * @param jaasClientEntry the name of the login config entry
+   * @param principal principal of the client.
+   * @param keytab location to the keytab file
+   * @return a registry service client instance
+   */
+  public static RegistryOperations createKerberosInstance(Configuration conf,
+      String jaasClientEntry, String principal, String keytab) {
+    Preconditions.checkArgument(conf != null, "Null configuration");
+    conf.set(KEY_REGISTRY_CLIENT_AUTH, REGISTRY_CLIENT_AUTH_KERBEROS);
+    conf.set(KEY_REGISTRY_CLIENT_JAAS_CONTEXT, jaasClientEntry);
+    RegistryOperationsClient operations =
+        new RegistryOperationsClient("KerberosRegistryOperations");
+    operations.setKerberosPrincipalAndKeytab(principal, keytab);
+    operations.init(conf);
+    return operations;
+  }
+
+
+  /**
    * Create and initialize an operations instance authenticated with write
    * access via an <code>id:password</code> pair.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/CuratorService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/CuratorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/CuratorService.java
index 8713920..c81a0ee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/CuratorService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/CuratorService.java
@@ -127,6 +127,7 @@ public class CuratorService extends CompositeService
     } else {
       this.bindingSource = this;
     }
+    registrySecurity = new RegistrySecurity("registry security");
   }
 
   /**
@@ -152,8 +153,7 @@ public class CuratorService extends CompositeService
     registryRoot = conf.getTrimmed(KEY_REGISTRY_ZK_ROOT,
         DEFAULT_ZK_REGISTRY_ROOT);
 
-    // create and add the registy service
-    registrySecurity = new RegistrySecurity("registry security");
+    // add the registy service
     addService(registrySecurity);
 
     if (LOG.isDebugEnabled()) {
@@ -163,6 +163,10 @@ public class CuratorService extends CompositeService
     super.serviceInit(conf);
   }
 
+  public void setKerberosPrincipalAndKeytab(String principal, String keytab) {
+    registrySecurity.setKerberosPrincipalAndKeytab(principal, keytab);
+  }
+
   /**
    * Start the service.
    * This is where the curator instance is started.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
index 23fadb5..ff6e8aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
@@ -23,6 +23,7 @@ import com.google.common.base.Splitter;
 import com.google.common.collect.Lists;
 import org.apache.commons.lang.StringUtils;
 import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.util.KerberosUtil;
@@ -31,6 +32,7 @@ import org.apache.hadoop.service.ServiceStateException;
 import org.apache.hadoop.util.ZKUtil;
 import org.apache.zookeeper.Environment;
 import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.client.ZooKeeperSaslClient;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
@@ -44,9 +46,11 @@ import java.lang.reflect.InvocationTargetException;
 import java.security.NoSuchAlgorithmException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.ListIterator;
 import java.util.Locale;
+import java.util.Map;
 import java.util.concurrent.CopyOnWriteArrayList;
 
 import static org.apache.hadoop.registry.client.impl.zk.ZookeeperConfigOptions.*;
@@ -170,13 +174,17 @@ public class RegistrySecurity extends AbstractService {
   /**
    * Client context
    */
-  private String jaasClientContext;
+  private String jaasClientEntry;
 
   /**
    * Client identity
    */
   private String jaasClientIdentity;
 
+  private String principal;
+
+  private String keytab;
+
   /**
    * Create an instance
    * @param name service name
@@ -238,6 +246,8 @@ public class RegistrySecurity extends AbstractService {
 
       systemACLs.addAll(buildACLs(system, kerberosRealm, ZooDefs.Perms.ALL));
 
+      LOG.info("Registry default system acls: " + System.lineSeparator() +
+          systemACLs);
       // user accounts (may be empty, but for digest one user AC must
       // be built up
       String user = getConfig().get(KEY_REGISTRY_USER_ACCOUNTS,
@@ -252,6 +262,7 @@ public class RegistrySecurity extends AbstractService {
           userACLs.add(self);
         }
       }
+      LOG.info("Registry User ACLs " + System.lineSeparator()+ userACLs);
 
       // here check for UGI having secure on or digest + ID
       switch (access) {
@@ -262,13 +273,12 @@ public class RegistrySecurity extends AbstractService {
           }
           UserGroupInformation currentUser =
               UserGroupInformation.getCurrentUser();
-          jaasClientContext = getOrFail(KEY_REGISTRY_CLIENT_JAAS_CONTEXT,
+          jaasClientEntry = getOrFail(KEY_REGISTRY_CLIENT_JAAS_CONTEXT,
               DEFAULT_REGISTRY_CLIENT_JAAS_CONTEXT);
           jaasClientIdentity = currentUser.getShortUserName();
           if (LOG.isDebugEnabled()) {
             LOG.debug("Auth is SASL user=\"{}\" JAAS context=\"{}\"",
-                jaasClientIdentity,
-                jaasClientContext);
+                jaasClientIdentity, jaasClientEntry);
           }
           break;
 
@@ -738,9 +748,81 @@ public class RegistrySecurity extends AbstractService {
           break;
 
         case sasl:
-          // bind to the current identity and context within the JAAS file
-          setZKSaslClientProperties(jaasClientIdentity, jaasClientContext);
+          JaasConfiguration jconf =
+              new JaasConfiguration(jaasClientEntry, principal, keytab);
+          javax.security.auth.login.Configuration.setConfiguration(jconf);
+          setSystemPropertyIfUnset(ZooKeeperSaslClient.ENABLE_CLIENT_SASL_KEY,
+              "true");
+          setSystemPropertyIfUnset(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY,
+              jaasClientEntry);
+          LOG.info(
+              "Enabling ZK sasl client: jaasClientEntry = " + jaasClientEntry
+                  + ", principal = " + principal + ", keytab = " + keytab);
+      }
+    }
+  }
+
+  public void setKerberosPrincipalAndKeytab(String principal, String keytab) {
+    this.principal = principal;
+    this.keytab = keytab;
+  }
+
+  /**
+   * Creates a programmatic version of a jaas.conf file. This can be used
+   * instead of writing a jaas.conf file and setting the system property,
+   * "java.security.auth.login.config", to point to that file. It is meant to be
+   * used for connecting to ZooKeeper.
+   */
+  @InterfaceAudience.Private
+  public static class JaasConfiguration extends
+      javax.security.auth.login.Configuration {
+
+    private final javax.security.auth.login.Configuration baseConfig =
+        javax.security.auth.login.Configuration.getConfiguration();
+    private static AppConfigurationEntry[] entry;
+    private String entryName;
+
+    /**
+     * Add an entry to the jaas configuration with the passed in name,
+     * principal, and keytab. The other necessary options will be set for you.
+     *
+     * @param entryName The name of the entry (e.g. "Client")
+     * @param principal The principal of the user
+     * @param keytab The location of the keytab
+     */
+    public JaasConfiguration(String entryName, String principal, String keytab) {
+      this.entryName = entryName;
+      Map<String, String> options = new HashMap<String, String>();
+      options.put("keyTab", keytab);
+      options.put("principal", principal);
+      options.put("useKeyTab", "true");
+      options.put("storeKey", "true");
+      options.put("useTicketCache", "false");
+      options.put("refreshKrb5Config", "true");
+      String jaasEnvVar = System.getenv("HADOOP_JAAS_DEBUG");
+      if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) {
+        options.put("debug", "true");
+      }
+      entry = new AppConfigurationEntry[]{
+          new AppConfigurationEntry(getKrb5LoginModuleName(),
+              AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
+              options)};
+    }
+
+    @Override
+    public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+      return (entryName.equals(name)) ? entry : ((baseConfig != null)
+          ? baseConfig.getAppConfigurationEntry(name) : null);
+    }
+
+    private String getKrb5LoginModuleName() {
+      String krb5LoginModuleName;
+      if (System.getProperty("java.vendor").contains("IBM")) {
+        krb5LoginModuleName = "com.ibm.security.auth.module.Krb5LoginModule";
+      } else {
+        krb5LoginModuleName = "com.sun.security.auth.module.Krb5LoginModule";
       }
+      return krb5LoginModuleName;
     }
   }
 
@@ -899,7 +981,7 @@ public class RegistrySecurity extends AbstractService {
              .append("; ");
       builder.append(KEY_REGISTRY_CLIENT_JAAS_CONTEXT)
              .append("=")
-             .append(jaasClientContext)
+             .append(jaasClientEntry)
              .append("; ");
       builder.append(describeProperty(PROP_ZK_SASL_CLIENT_USERNAME));
       builder.append(describeProperty(PROP_ZK_SASL_CLIENT_CONTEXT));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/dns/RegistryDNS.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/dns/RegistryDNS.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/dns/RegistryDNS.java
index d7ebece..358a963 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/dns/RegistryDNS.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/dns/RegistryDNS.java
@@ -412,6 +412,10 @@ public class RegistryDNS extends AbstractService implements DNSOperations,
       // Single reverse zone
     } else {
       Name reverseLookupZoneName = getReverseZoneName(conf);
+      if (reverseLookupZoneName == null) {
+        // reverse lookup disabled
+        return;
+      }
       Zone reverseLookupZone = configureZone(reverseLookupZoneName, conf);
       zones.put(reverseLookupZone.getOrigin(), reverseLookupZone);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/integration/RMRegistryOperationsService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/integration/RMRegistryOperationsService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/integration/RMRegistryOperationsService.java
deleted file mode 100644
index e11890f..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/integration/RMRegistryOperationsService.java
+++ /dev/null
@@ -1,246 +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.registry.server.integration;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.curator.framework.api.BackgroundCallback;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.registry.client.impl.zk.RegistryBindingSource;
-import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
-import org.apache.hadoop.registry.server.services.DeleteCompletionCallback;
-import org.apache.hadoop.registry.server.services.RegistryAdminService;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.concurrent.Future;
-
-/**
- * Handle RM events by updating the registry
- * <p>
- * These actions are all implemented as event handlers to operations
- * which come from the RM.
- * <p>
- * This service is expected to be executed by a user with the permissions
- * to manipulate the entire registry,
- */
-@InterfaceAudience.LimitedPrivate("YARN")
-@InterfaceStability.Evolving
-public class RMRegistryOperationsService extends RegistryAdminService {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(RMRegistryOperationsService.class);
-
-  private PurgePolicy purgeOnCompletionPolicy = PurgePolicy.PurgeAll;
-
-  public RMRegistryOperationsService(String name) {
-    this(name, null);
-  }
-
-  public RMRegistryOperationsService(String name,
-      RegistryBindingSource bindingSource) {
-    super(name, bindingSource);
-  }
-
-
-  /**
-   * Extend the parent service initialization by verifying that the
-   * service knows —in a secure cluster— the realm in which it is executing.
-   * It needs this to properly build up the user names and hence their
-   * access rights.
-   *
-   * @param conf configuration of the service
-   * @throws Exception
-   */
-  @Override
-  protected void serviceInit(Configuration conf) throws Exception {
-    super.serviceInit(conf);
-
-    verifyRealmValidity();
-  }
-
-  public PurgePolicy getPurgeOnCompletionPolicy() {
-    return purgeOnCompletionPolicy;
-  }
-
-  public void setPurgeOnCompletionPolicy(PurgePolicy purgeOnCompletionPolicy) {
-    this.purgeOnCompletionPolicy = purgeOnCompletionPolicy;
-  }
-
-  public void onApplicationAttemptRegistered(ApplicationAttemptId attemptId,
-      String host, int rpcport, String trackingurl) throws IOException {
-
-  }
-
-  public void onApplicationLaunched(ApplicationId id) throws IOException {
-
-  }
-
-  /**
-   * Actions to take as an AM registers itself with the RM.
-   * @param attemptId attempt ID
-   * @throws IOException problems
-   */
-  public void onApplicationMasterRegistered(ApplicationAttemptId attemptId) throws
-      IOException {
-  }
-
-  /**
-   * Actions to take when the AM container is completed
-   * @param containerId  container ID
-   * @throws IOException problems
-   */
-  public void onAMContainerFinished(ContainerId containerId) throws
-      IOException {
-    LOG.info("AM Container {} finished, purging application attempt records",
-        containerId);
-
-    // remove all application attempt entries
-    purgeAppAttemptRecords(containerId.getApplicationAttemptId());
-
-    // also treat as a container finish to remove container
-    // level records for the AM container
-    onContainerFinished(containerId);
-  }
-
-  /**
-   * remove all application attempt entries
-   * @param attemptId attempt ID
-   */
-  protected void purgeAppAttemptRecords(ApplicationAttemptId attemptId) {
-    purgeRecordsAsync("/",
-        attemptId.toString(),
-        PersistencePolicies.APPLICATION_ATTEMPT);
-  }
-
-  /**
-   * Actions to take when an application attempt is completed
-   * @param attemptId  application  ID
-   * @throws IOException problems
-   */
-  public void onApplicationAttemptUnregistered(ApplicationAttemptId attemptId)
-      throws IOException {
-    LOG.info("Application attempt {} unregistered, purging app attempt records",
-        attemptId);
-    purgeAppAttemptRecords(attemptId);
-  }
-
-  /**
-   * Actions to take when an application is completed
-   * @param id  application  ID
-   * @throws IOException problems
-   */
-  public void onApplicationCompleted(ApplicationId id)
-      throws IOException {
-    LOG.info("Application {} completed, purging application-level records",
-        id);
-    purgeRecordsAsync("/",
-        id.toString(),
-        PersistencePolicies.APPLICATION);
-  }
-
-  public void onApplicationAttemptAdded(ApplicationAttemptId appAttemptId) {
-  }
-
-  /**
-   * This is the event where the user is known, so the user directory
-   * can be created
-   * @param applicationId application  ID
-   * @param user username
-   * @throws IOException problems
-   */
-  public void onStateStoreEvent(ApplicationId applicationId, String user) throws
-      IOException {
-    initUserRegistryAsync(user);
-  }
-
-  /**
-   * Actions to take when the AM container is completed
-   * @param id  container ID
-   * @throws IOException problems
-   */
-  public void onContainerFinished(ContainerId id) throws IOException {
-    LOG.info("Container {} finished, purging container-level records",
-        id);
-    purgeRecordsAsync("/",
-        id.toString(),
-        PersistencePolicies.CONTAINER);
-  }
-
-  /**
-   * Queue an async operation to purge all matching records under a base path.
-   * <ol>
-   *   <li>Uses a depth first search</li>
-   *   <li>A match is on ID and persistence policy, or, if policy==-1, any match</li>
-   *   <li>If a record matches then it is deleted without any child searches</li>
-   *   <li>Deletions will be asynchronous if a callback is provided</li>
-   * </ol>
-   * @param path base path
-   * @param id ID for service record.id
-   * @param persistencePolicyMatch ID for the persistence policy to match:
-   * no match, no delete.
-   * @return a future that returns the #of records deleted
-   */
-  @VisibleForTesting
-  public Future<Integer> purgeRecordsAsync(String path,
-      String id,
-      String persistencePolicyMatch) {
-
-    return purgeRecordsAsync(path,
-        id, persistencePolicyMatch,
-        purgeOnCompletionPolicy,
-        new DeleteCompletionCallback());
-  }
-
-  /**
-   * Queue an async operation to purge all matching records under a base path.
-   * <ol>
-   *   <li>Uses a depth first search</li>
-   *   <li>A match is on ID and persistence policy, or, if policy==-1, any match</li>
-   *   <li>If a record matches then it is deleted without any child searches</li>
-   *   <li>Deletions will be asynchronous if a callback is provided</li>
-   * </ol>
-   * @param path base path
-   * @param id ID for service record.id
-   * @param persistencePolicyMatch ID for the persistence policy to match:
-   * no match, no delete.
-   * @param purgePolicy how to react to children under the entry
-   * @param callback an optional callback
-   * @return a future that returns the #of records deleted
-   */
-  @VisibleForTesting
-  public Future<Integer> purgeRecordsAsync(String path,
-      String id,
-      String persistencePolicyMatch,
-      PurgePolicy purgePolicy,
-      BackgroundCallback callback) {
-    LOG.info(" records under {} with ID {} and policy {}: {}",
-        path, id, persistencePolicyMatch);
-    return submit(
-        new AsyncPurge(path,
-            new SelectByYarnPersistence(id, persistencePolicyMatch),
-            purgePolicy,
-            callback));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/DeleteCompletionCallback.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/DeleteCompletionCallback.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/DeleteCompletionCallback.java
index e160d4a..829ef68 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/DeleteCompletionCallback.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/services/DeleteCompletionCallback.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.registry.server.services;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.api.BackgroundCallback;
 import org.apache.curator.framework.api.CuratorEvent;
-import org.apache.hadoop.registry.server.integration.RMRegistryOperationsService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -34,7 +33,7 @@ import java.util.concurrent.atomic.AtomicInteger;
  */
 public class DeleteCompletionCallback implements BackgroundCallback {
   private static final Logger LOG =
-      LoggerFactory.getLogger(RMRegistryOperationsService.class);
+      LoggerFactory.getLogger(DeleteCompletionCallback.class);
 
   private AtomicInteger events = new AtomicInteger(0);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/AbstractRegistryTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/AbstractRegistryTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/AbstractRegistryTest.java
index 5b34f60..0d4a467 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/AbstractRegistryTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/AbstractRegistryTest.java
@@ -23,7 +23,8 @@ import org.apache.hadoop.registry.client.api.RegistryOperations;
 import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
 import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
 import org.apache.hadoop.registry.client.types.ServiceRecord;
-import org.apache.hadoop.registry.server.integration.RMRegistryOperationsService;
+
+import org.apache.hadoop.registry.server.services.RegistryAdminService;
 import org.junit.Before;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -31,22 +32,16 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.net.URISyntaxException;
 
-/**
- * Abstract registry tests .. inits the field {@link #registry}
- * before the test with an instance of {@link RMRegistryOperationsService};
- * and {@link #operations} with the same instance cast purely
- * to the type {@link RegistryOperations}.
- *
- */
+
 public class AbstractRegistryTest extends AbstractZKRegistryTest {
   private static final Logger LOG =
       LoggerFactory.getLogger(AbstractRegistryTest.class);
-  protected RMRegistryOperationsService registry;
+  protected RegistryAdminService registry;
   protected RegistryOperations operations;
 
   @Before
   public void setupRegistry() throws IOException {
-    registry = new RMRegistryOperationsService("yarnRegistry");
+    registry = new RegistryAdminService("yarnRegistry");
     operations = registry;
     registry.init(createRegistryConfiguration());
     registry.start();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/integration/TestRegistryRMOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/integration/TestRegistryRMOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/integration/TestRegistryRMOperations.java
deleted file mode 100644
index 451a69b..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/integration/TestRegistryRMOperations.java
+++ /dev/null
@@ -1,369 +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.registry.integration;
-
-import org.apache.curator.framework.api.BackgroundCallback;
-import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
-import org.apache.hadoop.registry.AbstractRegistryTest;
-import org.apache.hadoop.registry.client.api.BindFlags;
-import org.apache.hadoop.registry.client.api.RegistryConstants;
-import org.apache.hadoop.registry.client.binding.RegistryUtils;
-import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
-import org.apache.hadoop.registry.client.impl.zk.ZKPathDumper;
-import org.apache.hadoop.registry.client.impl.CuratorEventCatcher;
-import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
-import org.apache.hadoop.registry.client.types.RegistryPathStatus;
-import org.apache.hadoop.registry.client.types.ServiceRecord;
-import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
-import org.apache.hadoop.registry.server.services.DeleteCompletionCallback;
-import org.apache.hadoop.registry.server.services.RegistryAdminService;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-
-import static org.apache.hadoop.registry.client.binding.RegistryTypeUtils.inetAddrEndpoint;
-import static org.apache.hadoop.registry.client.binding.RegistryTypeUtils.restEndpoint;
-
-public class TestRegistryRMOperations extends AbstractRegistryTest {
-  protected static final Logger LOG =
-      LoggerFactory.getLogger(TestRegistryRMOperations.class);
-
-  /**
-   * trigger a purge operation
-   * @param path path
-   * @param id yarn ID
-   * @param policyMatch policy to match ID on
-   * @param purgePolicy policy when there are children under a match
-   * @return the number purged
-   * @throws IOException
-   */
-  public int purge(String path,
-      String id,
-      String policyMatch,
-      RegistryAdminService.PurgePolicy purgePolicy) throws
-      IOException,
-      ExecutionException,
-      InterruptedException {
-    return purge(path, id, policyMatch, purgePolicy, null);
-  }
-
-  /**
-   *
-   * trigger a purge operation
-   * @param path pathn
-   * @param id yarn ID
-   * @param policyMatch policy to match ID on
-   * @param purgePolicy policy when there are children under a match
-   * @param callback optional callback
-   * @return the number purged
-   * @throws IOException
-   */
-  public int purge(String path,
-      String id,
-      String policyMatch,
-      RegistryAdminService.PurgePolicy purgePolicy,
-      BackgroundCallback callback) throws
-      IOException,
-      ExecutionException,
-      InterruptedException {
-
-    Future<Integer> future = registry.purgeRecordsAsync(path,
-        id, policyMatch, purgePolicy, callback);
-    try {
-      return future.get();
-    } catch (ExecutionException e) {
-      if (e.getCause() instanceof IOException) {
-        throw (IOException) e.getCause();
-      } else {
-        throw e;
-      }
-    }
-  }
-
-  @Test
-  public void testPurgeEntryCuratorCallback() throws Throwable {
-
-    String path = "/users/example/hbase/hbase1/";
-    ServiceRecord written = buildExampleServiceEntry(
-        PersistencePolicies.APPLICATION_ATTEMPT);
-    written.set(YarnRegistryAttributes.YARN_ID,
-        "testAsyncPurgeEntry_attempt_001");
-
-    operations.mknode(RegistryPathUtils.parentOf(path), true);
-    operations.bind(path, written, 0);
-
-    ZKPathDumper dump = registry.dumpPath(false);
-    CuratorEventCatcher events = new CuratorEventCatcher();
-
-    LOG.info("Initial state {}", dump);
-
-    // container query
-    String id = written.get(YarnRegistryAttributes.YARN_ID, "");
-    int opcount = purge("/",
-        id,
-        PersistencePolicies.CONTAINER,
-        RegistryAdminService.PurgePolicy.PurgeAll,
-        events);
-    assertPathExists(path);
-    assertEquals(0, opcount);
-    assertEquals("Event counter", 0, events.getCount());
-
-    // now the application attempt
-    opcount = purge("/",
-        id,
-        PersistencePolicies.APPLICATION_ATTEMPT,
-        RegistryAdminService.PurgePolicy.PurgeAll,
-        events);
-
-    LOG.info("Final state {}", dump);
-
-    assertPathNotFound(path);
-    assertEquals("wrong no of delete operations in " + dump, 1, opcount);
-    // and validate the callback event
-    assertEquals("Event counter", 1, events.getCount());
-  }
-
-  @Test
-  public void testAsyncPurgeEntry() throws Throwable {
-
-    String path = "/users/example/hbase/hbase1/";
-    ServiceRecord written = buildExampleServiceEntry(
-        PersistencePolicies.APPLICATION_ATTEMPT);
-    written.set(YarnRegistryAttributes.YARN_ID,
-        "testAsyncPurgeEntry_attempt_001");
-
-    operations.mknode(RegistryPathUtils.parentOf(path), true);
-    operations.bind(path, written, 0);
-
-    ZKPathDumper dump = registry.dumpPath(false);
-
-    LOG.info("Initial state {}", dump);
-
-    DeleteCompletionCallback deletions = new DeleteCompletionCallback();
-    int opcount = purge("/",
-        written.get(YarnRegistryAttributes.YARN_ID, ""),
-        PersistencePolicies.CONTAINER,
-        RegistryAdminService.PurgePolicy.PurgeAll,
-        deletions);
-    assertPathExists(path);
-
-    dump = registry.dumpPath(false);
-
-    assertEquals("wrong no of delete operations in " + dump, 0,
-        deletions.getEventCount());
-    assertEquals("wrong no of delete operations in " + dump, 0, opcount);
-
-
-    // now app attempt
-    deletions = new DeleteCompletionCallback();
-    opcount = purge("/",
-        written.get(YarnRegistryAttributes.YARN_ID, ""),
-        PersistencePolicies.APPLICATION_ATTEMPT,
-        RegistryAdminService.PurgePolicy.PurgeAll,
-        deletions);
-
-    dump = registry.dumpPath(false);
-    LOG.info("Final state {}", dump);
-
-    assertPathNotFound(path);
-    assertEquals("wrong no of delete operations in " + dump, 1,
-        deletions.getEventCount());
-    assertEquals("wrong no of delete operations in " + dump, 1, opcount);
-    // and validate the callback event
-
-  }
-
-  @Test
-  public void testPutGetContainerPersistenceServiceEntry() throws Throwable {
-
-    String path = ENTRY_PATH;
-    ServiceRecord written = buildExampleServiceEntry(
-        PersistencePolicies.CONTAINER);
-
-    operations.mknode(RegistryPathUtils.parentOf(path), true);
-    operations.bind(path, written, BindFlags.CREATE);
-    ServiceRecord resolved = operations.resolve(path);
-    validateEntry(resolved);
-    assertMatches(written, resolved);
-  }
-
-  /**
-   * Create a complex example app
-   * @throws Throwable
-   */
-  @Test
-  public void testCreateComplexApplication() throws Throwable {
-    String appId = "application_1408631738011_0001";
-    String cid = "container_1408631738011_0001_01_";
-    String cid1 = cid + "000001";
-    String cid2 = cid + "000002";
-    String appPath = USERPATH + "tomcat";
-
-    ServiceRecord webapp = createRecord(appId,
-        PersistencePolicies.APPLICATION, "tomcat-based web application",
-        null);
-    webapp.addExternalEndpoint(restEndpoint("www",
-        new URI("http", "//loadbalancer/", null)));
-
-    ServiceRecord comp1 = createRecord(cid1, PersistencePolicies.CONTAINER,
-        null,
-        null);
-    comp1.addExternalEndpoint(restEndpoint("www",
-        new URI("http", "//rack4server3:43572", null)));
-    comp1.addInternalEndpoint(
-        inetAddrEndpoint("jmx", "JMX", "rack4server3", 43573));
-
-    // Component 2 has a container lifespan
-    ServiceRecord comp2 = createRecord(cid2, PersistencePolicies.CONTAINER,
-        null,
-        null);
-    comp2.addExternalEndpoint(restEndpoint("www",
-        new URI("http", "//rack1server28:35881", null)));
-    comp2.addInternalEndpoint(
-        inetAddrEndpoint("jmx", "JMX", "rack1server28", 35882));
-
-    operations.mknode(USERPATH, false);
-    operations.bind(appPath, webapp, BindFlags.OVERWRITE);
-    String componentsPath = appPath + RegistryConstants.SUBPATH_COMPONENTS;
-    operations.mknode(componentsPath, false);
-    String dns1 = RegistryPathUtils.encodeYarnID(cid1);
-    String dns1path = componentsPath + dns1;
-    operations.bind(dns1path, comp1, BindFlags.CREATE);
-    String dns2 = RegistryPathUtils.encodeYarnID(cid2);
-    String dns2path = componentsPath + dns2;
-    operations.bind(dns2path, comp2, BindFlags.CREATE);
-
-    ZKPathDumper pathDumper = registry.dumpPath(false);
-    LOG.info(pathDumper.toString());
-
-    logRecord("tomcat", webapp);
-    logRecord(dns1, comp1);
-    logRecord(dns2, comp2);
-
-    ServiceRecord dns1resolved = operations.resolve(dns1path);
-    assertEquals("Persistence policies on resolved entry",
-        PersistencePolicies.CONTAINER,
-        dns1resolved.get(YarnRegistryAttributes.YARN_PERSISTENCE, ""));
-
-    Map<String, RegistryPathStatus> children =
-        RegistryUtils.statChildren(operations, componentsPath);
-    assertEquals(2, children.size());
-    Collection<RegistryPathStatus>
-        componentStats = children.values();
-    Map<String, ServiceRecord> records =
-        RegistryUtils.extractServiceRecords(operations,
-            componentsPath, componentStats);
-    assertEquals(2, records.size());
-    ServiceRecord retrieved1 = records.get(dns1path);
-    logRecord(retrieved1.get(YarnRegistryAttributes.YARN_ID, ""), retrieved1);
-    assertMatches(dns1resolved, retrieved1);
-    assertEquals(PersistencePolicies.CONTAINER,
-        retrieved1.get(YarnRegistryAttributes.YARN_PERSISTENCE, ""));
-
-    // create a listing under components/
-    operations.mknode(componentsPath + "subdir", false);
-
-    // this shows up in the listing of child entries
-    Map<String, RegistryPathStatus> childrenUpdated =
-        RegistryUtils.statChildren(operations, componentsPath);
-    assertEquals(3, childrenUpdated.size());
-
-    // the non-record child this is not picked up in the record listing
-    Map<String, ServiceRecord> recordsUpdated =
-
-        RegistryUtils.extractServiceRecords(operations,
-            componentsPath,
-            childrenUpdated);
-    assertEquals(2, recordsUpdated.size());
-
-    // now do some deletions.
-
-    // synchronous delete container ID 2
-
-    // fail if the app policy is chosen
-    assertEquals(0, purge("/", cid2, PersistencePolicies.APPLICATION,
-        RegistryAdminService.PurgePolicy.FailOnChildren));
-    // succeed for container
-    assertEquals(1, purge("/", cid2, PersistencePolicies.CONTAINER,
-        RegistryAdminService.PurgePolicy.FailOnChildren));
-    assertPathNotFound(dns2path);
-    assertPathExists(dns1path);
-
-    // expect a skip on children to skip
-    assertEquals(0,
-        purge("/", appId, PersistencePolicies.APPLICATION,
-            RegistryAdminService.PurgePolicy.SkipOnChildren));
-    assertPathExists(appPath);
-    assertPathExists(dns1path);
-
-    // attempt to delete app with policy of fail on children
-    try {
-      int p = purge("/",
-          appId,
-          PersistencePolicies.APPLICATION,
-          RegistryAdminService.PurgePolicy.FailOnChildren);
-      fail("expected a failure, got a purge count of " + p);
-    } catch (PathIsNotEmptyDirectoryException expected) {
-      // expected
-    }
-    assertPathExists(appPath);
-    assertPathExists(dns1path);
-
-
-    // now trigger recursive delete
-    assertEquals(1,
-        purge("/", appId, PersistencePolicies.APPLICATION,
-            RegistryAdminService.PurgePolicy.PurgeAll));
-    assertPathNotFound(appPath);
-    assertPathNotFound(dns1path);
-
-  }
-
-  @Test
-  public void testChildDeletion() throws Throwable {
-    ServiceRecord app = createRecord("app1",
-        PersistencePolicies.APPLICATION, "app",
-        null);
-    ServiceRecord container = createRecord("container1",
-        PersistencePolicies.CONTAINER, "container",
-        null);
-
-    operations.bind("/app", app, BindFlags.OVERWRITE);
-    operations.bind("/app/container", container, BindFlags.OVERWRITE);
-
-    try {
-      int p = purge("/",
-          "app1",
-          PersistencePolicies.APPLICATION,
-          RegistryAdminService.PurgePolicy.FailOnChildren);
-      fail("expected a failure, got a purge count of " + p);
-    } catch (PathIsNotEmptyDirectoryException expected) {
-      // expected
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestSecureRMRegistryOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestSecureRMRegistryOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestSecureRMRegistryOperations.java
deleted file mode 100644
index 41760d6..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/test/java/org/apache/hadoop/registry/secure/TestSecureRMRegistryOperations.java
+++ /dev/null
@@ -1,348 +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.registry.secure;
-
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.PathPermissionException;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.service.ServiceStateException;
-import org.apache.hadoop.registry.client.api.RegistryConstants;
-import org.apache.hadoop.registry.client.api.RegistryOperations;
-import org.apache.hadoop.registry.client.api.RegistryOperationsFactory;
-import org.apache.hadoop.registry.client.exceptions.NoPathPermissionsException;
-import org.apache.hadoop.registry.client.impl.zk.ZKPathDumper;
-import org.apache.hadoop.registry.client.impl.RegistryOperationsClient;
-import org.apache.hadoop.registry.client.impl.zk.RegistrySecurity;
-import org.apache.hadoop.registry.client.impl.zk.ZookeeperConfigOptions;
-import org.apache.hadoop.registry.server.integration.RMRegistryOperationsService;
-import org.apache.hadoop.registry.server.services.RegistryAdminService;
-import org.apache.zookeeper.client.ZooKeeperSaslClient;
-import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Id;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.security.auth.login.LoginException;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
-import java.util.List;
-
-import static org.apache.hadoop.registry.client.api.RegistryConstants.*;
-
-/**
- * Verify that the {@link RMRegistryOperationsService} works securely
- */
-public class TestSecureRMRegistryOperations extends AbstractSecureRegistryTest {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(TestSecureRMRegistryOperations.class);
-  private Configuration secureConf;
-  private Configuration zkClientConf;
-  private UserGroupInformation zookeeperUGI;
-
-  @Before
-  public void setupTestSecureRMRegistryOperations() throws Exception {
-    startSecureZK();
-    secureConf = new Configuration();
-    secureConf.setBoolean(KEY_REGISTRY_SECURE, true);
-
-    // create client conf containing the ZK quorum
-    zkClientConf = new Configuration(secureZK.getConfig());
-    zkClientConf.setBoolean(KEY_REGISTRY_SECURE, true);
-    assertNotEmpty(zkClientConf.get(RegistryConstants.KEY_REGISTRY_ZK_QUORUM));
-
-    // ZK is in charge
-    secureConf.set(KEY_REGISTRY_SYSTEM_ACCOUNTS, "sasl:zookeeper@");
-    zookeeperUGI = loginUGI(ZOOKEEPER, keytab_zk);
-  }
-
-  @After
-  public void teardownTestSecureRMRegistryOperations() {
-  }
-
-  /**
-   * Create the RM registry operations as the current user
-   * @return the service
-   * @throws LoginException
-   * @throws FileNotFoundException
-   */
-  public RMRegistryOperationsService startRMRegistryOperations() throws
-      LoginException, IOException, InterruptedException {
-    // kerberos
-    secureConf.set(KEY_REGISTRY_CLIENT_AUTH,
-        REGISTRY_CLIENT_AUTH_KERBEROS);
-    secureConf.set(KEY_REGISTRY_CLIENT_JAAS_CONTEXT, ZOOKEEPER_CLIENT_CONTEXT);
-
-    RMRegistryOperationsService registryOperations = zookeeperUGI.doAs(
-        new PrivilegedExceptionAction<RMRegistryOperationsService>() {
-          @Override
-          public RMRegistryOperationsService run() throws Exception {
-            RMRegistryOperationsService operations
-                = new RMRegistryOperationsService("rmregistry", secureZK);
-            addToTeardown(operations);
-            operations.init(secureConf);
-            LOG.info(operations.bindingDiagnosticDetails());
-            operations.start();
-            return operations;
-          }
-        });
-
-    return registryOperations;
-  }
-
-  /**
-   * test that ZK can write as itself
-   * @throws Throwable
-   */
-  @Test
-  public void testZookeeperCanWriteUnderSystem() throws Throwable {
-
-    RMRegistryOperationsService rmRegistryOperations =
-        startRMRegistryOperations();
-    RegistryOperations operations = rmRegistryOperations;
-    operations.mknode(PATH_SYSTEM_SERVICES + "hdfs",
-        false);
-    ZKPathDumper pathDumper = rmRegistryOperations.dumpPath(true);
-    LOG.info(pathDumper.toString());
-  }
-
-  @Test
-  public void testAnonReadAccess() throws Throwable {
-    RMRegistryOperationsService rmRegistryOperations =
-        startRMRegistryOperations();
-    describe(LOG, "testAnonReadAccess");
-    RegistryOperations operations =
-        RegistryOperationsFactory.createAnonymousInstance(zkClientConf);
-    addToTeardown(operations);
-    operations.start();
-
-    assertFalse("RegistrySecurity.isClientSASLEnabled()==true",
-        RegistrySecurity.isClientSASLEnabled());
-    operations.list(PATH_SYSTEM_SERVICES);
-  }
-
-  @Test
-  public void testAnonNoWriteAccess() throws Throwable {
-    RMRegistryOperationsService rmRegistryOperations =
-        startRMRegistryOperations();
-    describe(LOG, "testAnonNoWriteAccess");
-    RegistryOperations operations =
-        RegistryOperationsFactory.createAnonymousInstance(zkClientConf);
-    addToTeardown(operations);
-    operations.start();
-
-    String servicePath = PATH_SYSTEM_SERVICES + "hdfs";
-    expectMkNodeFailure(operations, servicePath);
-  }
-
-  @Test
-  public void testAnonNoWriteAccessOffRoot() throws Throwable {
-    RMRegistryOperationsService rmRegistryOperations =
-        startRMRegistryOperations();
-    describe(LOG, "testAnonNoWriteAccessOffRoot");
-    RegistryOperations operations =
-        RegistryOperationsFactory.createAnonymousInstance(zkClientConf);
-    addToTeardown(operations);
-    operations.start();
-    assertFalse("mknode(/)", operations.mknode("/", false));
-    expectMkNodeFailure(operations, "/sub");
-    expectDeleteFailure(operations, PATH_SYSTEM_SERVICES, true);
-  }
-
-  /**
-   * Expect a mknode operation to fail
-   * @param operations operations instance
-   * @param path path
-   * @throws IOException An IO failure other than those permitted
-   */
-  public void expectMkNodeFailure(RegistryOperations operations,
-      String path) throws IOException {
-    try {
-      operations.mknode(path, false);
-      fail("should have failed to create a node under " + path);
-    } catch (PathPermissionException expected) {
-      // expected
-    } catch (NoPathPermissionsException expected) {
-      // expected
-    }
-  }
-
-  /**
-   * Expect a delete operation to fail
-   * @param operations operations instance
-   * @param path path
-   * @param recursive
-   * @throws IOException An IO failure other than those permitted
-   */
-  public void expectDeleteFailure(RegistryOperations operations,
-      String path, boolean recursive) throws IOException {
-    try {
-      operations.delete(path, recursive);
-      fail("should have failed to delete the node " + path);
-    } catch (PathPermissionException expected) {
-      // expected
-    } catch (NoPathPermissionsException expected) {
-      // expected
-    }
-  }
-
-  @Test
-  public void testAlicePathRestrictedAnonAccess() throws Throwable {
-    RMRegistryOperationsService rmRegistryOperations =
-        startRMRegistryOperations();
-    String aliceHome = rmRegistryOperations.initUserRegistry(ALICE);
-    describe(LOG, "Creating anonymous accessor");
-    RegistryOperations anonOperations =
-        RegistryOperationsFactory.createAnonymousInstance(zkClientConf);
-    addToTeardown(anonOperations);
-    anonOperations.start();
-    anonOperations.list(aliceHome);
-    expectMkNodeFailure(anonOperations, aliceHome + "/anon");
-    expectDeleteFailure(anonOperations, aliceHome, true);
-  }
-
-  @Test
-  public void testUserZookeeperHomePathAccess() throws Throwable {
-    RMRegistryOperationsService rmRegistryOperations =
-        startRMRegistryOperations();
-    final String home = rmRegistryOperations.initUserRegistry(ZOOKEEPER);
-    describe(LOG, "Creating ZK client");
-
-    RegistryOperations operations = zookeeperUGI.doAs(
-        new PrivilegedExceptionAction<RegistryOperations>() {
-          @Override
-          public RegistryOperations run() throws Exception {
-            RegistryOperations operations =
-                RegistryOperationsFactory.createKerberosInstance(zkClientConf,
-                    ZOOKEEPER_CLIENT_CONTEXT);
-            addToTeardown(operations);
-            operations.start();
-
-            return operations;
-          }
-        });
-    operations.list(home);
-    String path = home + "/subpath";
-    operations.mknode(path, false);
-    operations.delete(path, true);
-  }
-
-  @Test
-  public void testUserHomedirsPermissionsRestricted() throws Throwable {
-    // test that the /users/$user permissions are restricted
-    RMRegistryOperationsService rmRegistryOperations =
-        startRMRegistryOperations();
-    // create Alice's dir, so it should have an ACL for Alice
-    final String home = rmRegistryOperations.initUserRegistry(ALICE);
-    List<ACL> acls = rmRegistryOperations.zkGetACLS(home);
-    ACL aliceACL = null;
-    for (ACL acl : acls) {
-      LOG.info(RegistrySecurity.aclToString(acl));
-      Id id = acl.getId();
-      if (id.getScheme().equals(ZookeeperConfigOptions.SCHEME_SASL)
-          && id.getId().startsWith(ALICE)) {
-
-        aliceACL = acl;
-        break;
-      }
-    }
-    assertNotNull(aliceACL);
-    assertEquals(RegistryAdminService.USER_HOMEDIR_ACL_PERMISSIONS,
-        aliceACL.getPerms());
-  }
-
-  @Test
-  public void testDigestAccess() throws Throwable {
-    RMRegistryOperationsService registryAdmin =
-        startRMRegistryOperations();
-    String id = "username";
-    String pass = "password";
-    registryAdmin.addWriteAccessor(id, pass);
-    List<ACL> clientAcls = registryAdmin.getClientAcls();
-    LOG.info("Client ACLS=\n{}", RegistrySecurity.aclsToString(clientAcls));
-
-    String base = "/digested";
-    registryAdmin.mknode(base, false);
-    List<ACL> baseACLs = registryAdmin.zkGetACLS(base);
-    String aclset = RegistrySecurity.aclsToString(baseACLs);
-    LOG.info("Base ACLs=\n{}", aclset);
-    ACL found = null;
-    for (ACL acl : baseACLs) {
-      if (ZookeeperConfigOptions.SCHEME_DIGEST.equals(acl.getId().getScheme())) {
-        found = acl;
-        break;
-      }
-    }
-    assertNotNull("Did not find digest entry in ACLs " + aclset, found);
-    zkClientConf.set(KEY_REGISTRY_USER_ACCOUNTS,
-        "sasl:somebody@EXAMPLE.COM, sasl:other");
-    RegistryOperations operations =
-        RegistryOperationsFactory.createAuthenticatedInstance(zkClientConf,
-            id,
-            pass);
-    addToTeardown(operations);
-    operations.start();
-    RegistryOperationsClient operationsClient =
-        (RegistryOperationsClient) operations;
-    List<ACL> digestClientACLs = operationsClient.getClientAcls();
-    LOG.info("digest client ACLs=\n{}",
-        RegistrySecurity.aclsToString(digestClientACLs));
-    operations.stat(base);
-    operations.mknode(base + "/subdir", false);
-    ZKPathDumper pathDumper = registryAdmin.dumpPath(true);
-    LOG.info(pathDumper.toString());
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testNoDigestAuthMissingId() throws Throwable {
-    RegistryOperationsFactory.createAuthenticatedInstance(zkClientConf,
-        "",
-        "pass");
-  }
-
-  @Test(expected = ServiceStateException.class)
-  public void testNoDigestAuthMissingId2() throws Throwable {
-    zkClientConf.set(KEY_REGISTRY_CLIENT_AUTH, REGISTRY_CLIENT_AUTH_DIGEST);
-    zkClientConf.set(KEY_REGISTRY_CLIENT_AUTHENTICATION_ID, "");
-    zkClientConf.set(KEY_REGISTRY_CLIENT_AUTHENTICATION_PASSWORD, "pass");
-    RegistryOperationsFactory.createInstance("DigestRegistryOperations",
-        zkClientConf);
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testNoDigestAuthMissingPass() throws Throwable {
-    RegistryOperationsFactory.createAuthenticatedInstance(zkClientConf,
-        "id",
-        "");
-  }
-
-  @Test(expected = ServiceStateException.class)
-  public void testNoDigestAuthMissingPass2() throws Throwable {
-    zkClientConf.set(KEY_REGISTRY_CLIENT_AUTH, REGISTRY_CLIENT_AUTH_DIGEST);
-    zkClientConf.set(KEY_REGISTRY_CLIENT_AUTHENTICATION_ID, "id");
-    zkClientConf.set(KEY_REGISTRY_CLIENT_AUTHENTICATION_PASSWORD, "");
-    RegistryOperationsFactory.createInstance("DigestRegistryOperations",
-        zkClientConf);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/QuickStart.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/QuickStart.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/QuickStart.md
index 512c011..f13d7d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/QuickStart.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/QuickStart.md
@@ -208,7 +208,25 @@ If you are building from source code, make sure you use `-Pyarn-ui` in the `mvn`
   </property>
 ```
 
-# Try with Docker
+# Run with security
+YARN service framework supports running in a secure(kerberized) environment. User needs to specify the kerberos principal name and keytab when they launch the service.
+E.g. A typical configuration looks like below:
+```
+{
+  "name": "sample-service",
+  ...
+  ...
+  "kerberos_principal" : {
+    "principal_name" : "hdfs-demo@EXAMPLE.COM",
+    "keytab" : "hdfs:///etc/security/keytabs/hdfs.headless.keytab"
+  }
+}
+```
+* principal_name : the principal name of the user who launches the service
+* keytab : URI of the keytab. It supports two modes:
+    * URI starts with `hdfs://`: The URI where the keytab is stored on hdfs. The keytab will be localized to each node by YARN.
+    * URI starts with `file://`: The URI where the keytab is stored on local host. It is assumed that admin pre-installs the keytabs on the local host before AM launches.
+# Run with Docker
 The above example is only for a non-docker container based service. YARN Service Framework also provides first-class support for managing docker based services.
 Most of the steps for managing docker based services are the same except that in docker the `Artifact` type for a component is `DOCKER` and the Artifact `id` is the name of the docker image.
 For details in how to setup docker on YARN, please check [Docker on YARN](../DockerContainers.md).

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d30d5782/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md
index c0e12c7..e224e5d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md
@@ -291,6 +291,15 @@ The current state of the container of a service.
 |----|----|----|----|----|
 |state|enum of the state of the container|false|enum (INIT, STARTED, READY)||
 
+### KerberosPrincipal
+
+The kerberos principal info of the user who launches the service.
+
+|Name|Description|Required|Schema|Default|
+|----|----|----|----|----|
+|principal_name|The principal name of the user who launches the service.|false|string||
+|keytab|The URI of the kerberos keytab. It supports two modes, URI starts with "hdfs://": A path on hdfs where the keytab is stored. The keytab will be localized by YARN to each host; URI starts with "file://": A path on the local host where the keytab is stored. It is assumed that the keytabs are pre-installed by admins before AM launches.|false|string||
+
 
 ### PlacementPolicy
 
@@ -342,7 +351,7 @@ a service resource has the following attributes.
 |state|State of the service. Specifying a value for this attribute for the PUT payload means update the service to this desired state.|false|ServiceState||
 |quicklinks|A blob of key-value pairs of quicklinks to be exported for a service.|false|object||
 |queue|The YARN queue that this service should be submitted to.|false|string||
-
+|kerberos_principal | The principal info of the user who launches the service|false||
 
 ### ServiceState
 


---------------------------------------------------------------------
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-12836. startTxId could be greater than endTxId when tailing in-progress edit log. Contributed by Chao Sun.

Posted by ae...@apache.org.
HDFS-12836. startTxId could be greater than endTxId when tailing in-progress edit log. Contributed by Chao Sun.


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

Branch: refs/heads/HDFS-7240
Commit: 0faf50624580b86b64a828cdbbb630ae8994e2cd
Parents: 53bbef3
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Fri Dec 1 12:01:21 2017 -0800
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Fri Dec 1 12:01:21 2017 -0800

----------------------------------------------------------------------
 .../qjournal/client/QuorumJournalManager.java    |  6 ++++++
 .../namenode/ha/TestStandbyInProgressTail.java   | 19 +++++++++++++++++++
 2 files changed, 25 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0faf5062/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
index d30625b..7dff9b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
@@ -498,6 +498,12 @@ public class QuorumJournalManager implements JournalManager {
         // than committedTxnId. This ensures the consistency.
         if (onlyDurableTxns && inProgressOk) {
           endTxId = Math.min(endTxId, committedTxnId);
+          if (endTxId < remoteLog.getStartTxId()) {
+            LOG.warn("Found endTxId (" + endTxId + ") that is less than " +
+                "the startTxId (" + remoteLog.getStartTxId() +
+                ") - setting it to startTxId.");
+            endTxId = remoteLog.getStartTxId();
+          }
         }
 
         EditLogInputStream elis = EditLogFileInputStream.fromUrl(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0faf5062/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyInProgressTail.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyInProgressTail.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyInProgressTail.java
index 9201cda..b1cd037 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyInProgressTail.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyInProgressTail.java
@@ -309,6 +309,25 @@ public class TestStandbyInProgressTail {
     assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test3", true));
   }
 
+  @Test
+  public void testNonUniformConfig() throws Exception {
+    // Test case where some NNs (in this case the active NN) in the cluster
+    // do not have in-progress tailing enabled.
+    Configuration newConf = cluster.getNameNode(0).getConf();
+    newConf.setBoolean(
+        DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY,
+        false);
+    cluster.restartNameNode(0);
+    cluster.transitionToActive(0);
+
+    cluster.getNameNode(0).getRpcServer().mkdirs("/test",
+        FsPermission.createImmutable((short) 0755), true);
+    cluster.getNameNode(0).getRpcServer().rollEdits();
+
+    cluster.getNameNode(1).getNamesystem().getEditLogTailer().doTailEdits();
+    assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test", true));
+  }
+
   /**
    * Check that no edits files are present in the given storage dirs.
    */


---------------------------------------------------------------------
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: MAPREDUCE-5124. AM lacks flow control for task events. Contributed by Peter Bacsko

Posted by ae...@apache.org.
MAPREDUCE-5124. AM lacks flow control for task events. Contributed by Peter Bacsko


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

Branch: refs/heads/HDFS-7240
Commit: 21d36273551fa45c4130e5523b6724358cf34b1e
Parents: 0faf506
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Dec 1 14:03:01 2017 -0600
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Dec 1 14:04:25 2017 -0600

----------------------------------------------------------------------
 .../hadoop/mapred/TaskAttemptListenerImpl.java  |  69 +++-
 .../job/event/TaskAttemptStatusUpdateEvent.java |  12 +-
 .../v2/app/job/impl/TaskAttemptImpl.java        |  20 +-
 .../mapred/TestTaskAttemptListenerImpl.java     | 315 ++++++++++++-------
 .../mapreduce/v2/app/TestFetchFailure.java      |   3 +-
 .../mapreduce/v2/app/TestMRClientService.java   |   4 +-
 .../v2/TestSpeculativeExecutionWithMRApp.java   |  13 +-
 7 files changed, 302 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/21d36273/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
index 9b6148c..67f8ff0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
@@ -22,9 +22,11 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.List;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -36,6 +38,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
 import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.TaskAttemptListener;
@@ -58,6 +61,8 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * This class is responsible for talking to the task umblical.
  * It also converts all the old data structures
@@ -66,7 +71,6 @@ import org.slf4j.LoggerFactory;
  * This class HAS to be in this package to access package private 
  * methods/classes.
  */
-@SuppressWarnings({"unchecked"})
 public class TaskAttemptListenerImpl extends CompositeService 
     implements TaskUmbilicalProtocol, TaskAttemptListener {
 
@@ -84,6 +88,11 @@ public class TaskAttemptListenerImpl extends CompositeService
   private ConcurrentMap<WrappedJvmID, org.apache.hadoop.mapred.Task>
     jvmIDToActiveAttemptMap
       = new ConcurrentHashMap<WrappedJvmID, org.apache.hadoop.mapred.Task>();
+
+  private ConcurrentMap<TaskAttemptId,
+      AtomicReference<TaskAttemptStatus>> attemptIdToStatus
+        = new ConcurrentHashMap<>();
+
   private Set<WrappedJvmID> launchedJVMs = Collections
       .newSetFromMap(new ConcurrentHashMap<WrappedJvmID, Boolean>());
 
@@ -359,6 +368,13 @@ public class TaskAttemptListenerImpl extends CompositeService
     org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId yarnAttemptID =
         TypeConverter.toYarn(taskAttemptID);
 
+    AtomicReference<TaskAttemptStatus> lastStatusRef =
+        attemptIdToStatus.get(yarnAttemptID);
+    if (lastStatusRef == null) {
+      throw new IllegalStateException("Status update was called"
+          + " with illegal TaskAttemptId: " + yarnAttemptID);
+    }
+
     AMFeedback feedback = new AMFeedback();
     feedback.setTaskFound(true);
 
@@ -437,9 +453,8 @@ public class TaskAttemptListenerImpl extends CompositeService
 //    // isn't ever changed by the Task itself.
 //    taskStatus.getIncludeCounters();
 
-    context.getEventHandler().handle(
-        new TaskAttemptStatusUpdateEvent(taskAttemptStatus.id,
-            taskAttemptStatus));
+    coalesceStatusUpdate(yarnAttemptID, taskAttemptStatus, lastStatusRef);
+
     return feedback;
   }
 
@@ -520,6 +535,8 @@ public class TaskAttemptListenerImpl extends CompositeService
     launchedJVMs.add(jvmId);
 
     taskHeartbeatHandler.register(attemptID);
+
+    attemptIdToStatus.put(attemptID, new AtomicReference<>());
   }
 
   @Override
@@ -541,6 +558,8 @@ public class TaskAttemptListenerImpl extends CompositeService
 
     //unregister this attempt
     taskHeartbeatHandler.unregister(attemptID);
+
+    attemptIdToStatus.remove(attemptID);
   }
 
   @Override
@@ -563,4 +582,46 @@ public class TaskAttemptListenerImpl extends CompositeService
     preemptionPolicy.setCheckpointID(tid, cid);
   }
 
+  private void coalesceStatusUpdate(TaskAttemptId yarnAttemptID,
+      TaskAttemptStatus taskAttemptStatus,
+      AtomicReference<TaskAttemptStatus> lastStatusRef) {
+    boolean asyncUpdatedNeeded = false;
+    TaskAttemptStatus lastStatus = lastStatusRef.get();
+
+    if (lastStatus == null) {
+      lastStatusRef.set(taskAttemptStatus);
+      asyncUpdatedNeeded = true;
+    } else {
+      List<TaskAttemptId> oldFetchFailedMaps =
+          taskAttemptStatus.fetchFailedMaps;
+
+      // merge fetchFailedMaps from the previous update
+      if (lastStatus.fetchFailedMaps != null) {
+        if (taskAttemptStatus.fetchFailedMaps == null) {
+          taskAttemptStatus.fetchFailedMaps = lastStatus.fetchFailedMaps;
+        } else {
+          taskAttemptStatus.fetchFailedMaps.addAll(lastStatus.fetchFailedMaps);
+        }
+      }
+
+      if (!lastStatusRef.compareAndSet(lastStatus, taskAttemptStatus)) {
+        // update failed - async dispatcher has processed it in the meantime
+        taskAttemptStatus.fetchFailedMaps = oldFetchFailedMaps;
+        lastStatusRef.set(taskAttemptStatus);
+        asyncUpdatedNeeded = true;
+      }
+    }
+
+    if (asyncUpdatedNeeded) {
+      context.getEventHandler().handle(
+          new TaskAttemptStatusUpdateEvent(taskAttemptStatus.id,
+              lastStatusRef));
+    }
+  }
+
+  @VisibleForTesting
+  ConcurrentMap<TaskAttemptId,
+      AtomicReference<TaskAttemptStatus>> getAttemptIdToStatus() {
+    return attemptIdToStatus;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21d36273/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskAttemptStatusUpdateEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskAttemptStatusUpdateEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskAttemptStatusUpdateEvent.java
index 715f63d..cef4fd0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskAttemptStatusUpdateEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/event/TaskAttemptStatusUpdateEvent.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.mapreduce.v2.app.job.event;
 
 import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.v2.api.records.Phase;
@@ -26,17 +27,16 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
 
 public class TaskAttemptStatusUpdateEvent extends TaskAttemptEvent {
-
-  private TaskAttemptStatus reportedTaskAttemptStatus;
+  private AtomicReference<TaskAttemptStatus> taskAttemptStatusRef;
 
   public TaskAttemptStatusUpdateEvent(TaskAttemptId id,
-      TaskAttemptStatus taskAttemptStatus) {
+      AtomicReference<TaskAttemptStatus> taskAttemptStatusRef) {
     super(id, TaskAttemptEventType.TA_UPDATE);
-    this.reportedTaskAttemptStatus = taskAttemptStatus;
+    this.taskAttemptStatusRef = taskAttemptStatusRef;
   }
 
-  public TaskAttemptStatus getReportedTaskAttemptStatus() {
-    return reportedTaskAttemptStatus;
+  public AtomicReference<TaskAttemptStatus> getTaskAttemptStatusRef() {
+    return taskAttemptStatusRef;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21d36273/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
index 90e0d21..431128b 100755
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
@@ -37,6 +37,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -1780,7 +1781,6 @@ public abstract class TaskAttemptImpl implements
     taskAttempt.updateProgressSplits();
   }
 
-
   static class RequestContainerTransition implements
       SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
     private final boolean rescheduled;
@@ -1965,6 +1965,7 @@ public abstract class TaskAttemptImpl implements
       // register it to TaskAttemptListener so that it can start monitoring it.
       taskAttempt.taskAttemptListener
         .registerLaunchedTask(taskAttempt.attemptId, taskAttempt.jvmID);
+
       //TODO Resolve to host / IP in case of a local address.
       InetSocketAddress nodeHttpInetAddr = // TODO: Costly to create sock-addr?
           NetUtils.createSocketAddr(taskAttempt.container.getNodeHttpAddress());
@@ -2430,15 +2431,20 @@ public abstract class TaskAttemptImpl implements
   }
 
   private static class StatusUpdater 
-       implements SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
+      implements SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
     @SuppressWarnings("unchecked")
     @Override
     public void transition(TaskAttemptImpl taskAttempt, 
         TaskAttemptEvent event) {
-      // Status update calls don't really change the state of the attempt.
+      TaskAttemptStatusUpdateEvent statusEvent =
+          ((TaskAttemptStatusUpdateEvent)event);
+
+      AtomicReference<TaskAttemptStatus> taskAttemptStatusRef =
+          statusEvent.getTaskAttemptStatusRef();
+
       TaskAttemptStatus newReportedStatus =
-          ((TaskAttemptStatusUpdateEvent) event)
-              .getReportedTaskAttemptStatus();
+          taskAttemptStatusRef.getAndSet(null);
+
       // Now switch the information in the reportedStatus
       taskAttempt.reportedStatus = newReportedStatus;
       taskAttempt.reportedStatus.taskState = taskAttempt.getState();
@@ -2447,12 +2453,10 @@ public abstract class TaskAttemptImpl implements
       taskAttempt.eventHandler.handle
           (new SpeculatorEvent
               (taskAttempt.reportedStatus, taskAttempt.clock.getTime()));
-      
       taskAttempt.updateProgressSplits();
-      
       //if fetch failures are present, send the fetch failure event to job
       //this only will happen in reduce attempt type
-      if (taskAttempt.reportedStatus.fetchFailedMaps != null && 
+      if (taskAttempt.reportedStatus.fetchFailedMaps != null &&
           taskAttempt.reportedStatus.fetchFailedMaps.size() > 0) {
         String hostname = taskAttempt.container == null ? "UNKNOWN"
             : taskAttempt.container.getNodeId().getHost();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21d36273/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
index fa8418a..4ff6fb2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
@@ -24,6 +24,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -35,6 +37,7 @@ import org.apache.hadoop.mapreduce.checkpoint.FSCheckpointID;
 import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
 import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.Phase;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEventStatus;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
@@ -42,6 +45,8 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.TaskHeartbeatHandler;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
 import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy;
 import org.apache.hadoop.mapreduce.v2.app.rm.preemption.CheckpointAMPreemptionPolicy;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
@@ -52,12 +57,69 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.util.SystemClock;
-
+import org.junit.After;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 
+/**
+ * Tests the behavior of TaskAttemptListenerImpl.
+ */
+@RunWith(MockitoJUnitRunner.class)
 public class TestTaskAttemptListenerImpl {
+  private static final String ATTEMPT1_ID =
+      "attempt_123456789012_0001_m_000001_0";
+  private static final String ATTEMPT2_ID =
+      "attempt_123456789012_0001_m_000002_0";
+
+  private static final TaskAttemptId TASKATTEMPTID1 =
+      TypeConverter.toYarn(TaskAttemptID.forName(ATTEMPT1_ID));
+  private static final TaskAttemptId TASKATTEMPTID2 =
+      TypeConverter.toYarn(TaskAttemptID.forName(ATTEMPT2_ID));
+
+  @Mock
+  private AppContext appCtx;
+
+  @Mock
+  private JobTokenSecretManager secret;
+
+  @Mock
+  private RMHeartbeatHandler rmHeartbeatHandler;
+
+  @Mock
+  private TaskHeartbeatHandler hbHandler;
+
+  @Mock
+  private Dispatcher dispatcher;
+
+  @Mock
+  private Task task;
+
+  @SuppressWarnings("rawtypes")
+  @Mock
+  private EventHandler<Event> ea;
+
+  @SuppressWarnings("rawtypes")
+  @Captor
+  private ArgumentCaptor<Event> eventCaptor;
+
+  private CheckpointAMPreemptionPolicy policy;
+  private JVMId id;
+  private WrappedJvmID wid;
+  private TaskAttemptID attemptID;
+  private TaskAttemptId attemptId;
+  private ReduceTaskStatus firstReduceStatus;
+  private ReduceTaskStatus secondReduceStatus;
+  private ReduceTaskStatus thirdReduceStatus;
+
+  private MockTaskAttemptListenerImpl listener;
+
   public static class MockTaskAttemptListenerImpl
       extends TaskAttemptListenerImpl {
 
@@ -93,34 +155,24 @@ public class TestTaskAttemptListenerImpl {
       //Empty
     }
   }
-  
+
+  @After
+  public void after() throws IOException {
+    if (listener != null) {
+      listener.close();
+      listener = null;
+    }
+  }
+
   @Test  (timeout=5000)
   public void testGetTask() throws IOException {
-    AppContext appCtx = mock(AppContext.class);
-    JobTokenSecretManager secret = mock(JobTokenSecretManager.class); 
-    RMHeartbeatHandler rmHeartbeatHandler =
-        mock(RMHeartbeatHandler.class);
-    TaskHeartbeatHandler hbHandler = mock(TaskHeartbeatHandler.class);
-    Dispatcher dispatcher = mock(Dispatcher.class);
-    @SuppressWarnings("unchecked")
-    EventHandler<Event> ea = mock(EventHandler.class);
-    when(dispatcher.getEventHandler()).thenReturn(ea);
-
-    when(appCtx.getEventHandler()).thenReturn(ea);
-    CheckpointAMPreemptionPolicy policy = new CheckpointAMPreemptionPolicy();
-    policy.init(appCtx);
-    MockTaskAttemptListenerImpl listener = 
-      new MockTaskAttemptListenerImpl(appCtx, secret,
-          rmHeartbeatHandler, hbHandler, policy);
-    Configuration conf = new Configuration();
-    listener.init(conf);
-    listener.start();
-    JVMId id = new JVMId("foo",1, true, 1);
-    WrappedJvmID wid = new WrappedJvmID(id.getJobId(), id.isMap, id.getId());
+    configureMocks();
+    startListener(false);
 
     // Verify ask before registration.
     //The JVM ID has not been registered yet so we should kill it.
     JvmContext context = new JvmContext();
+
     context.jvmId = id; 
     JvmTask result = listener.getTask(context);
     assertNotNull(result);
@@ -128,20 +180,18 @@ public class TestTaskAttemptListenerImpl {
 
     // Verify ask after registration but before launch. 
     // Don't kill, should be null.
-    TaskAttemptId attemptID = mock(TaskAttemptId.class);
-    Task task = mock(Task.class);
     //Now put a task with the ID
     listener.registerPendingTask(task, wid);
     result = listener.getTask(context);
     assertNull(result);
     // Unregister for more testing.
-    listener.unregister(attemptID, wid);
+    listener.unregister(attemptId, wid);
 
     // Verify ask after registration and launch
     //Now put a task with the ID
     listener.registerPendingTask(task, wid);
-    listener.registerLaunchedTask(attemptID, wid);
-    verify(hbHandler).register(attemptID);
+    listener.registerLaunchedTask(attemptId, wid);
+    verify(hbHandler).register(attemptId);
     result = listener.getTask(context);
     assertNotNull(result);
     assertFalse(result.shouldDie);
@@ -152,15 +202,13 @@ public class TestTaskAttemptListenerImpl {
     assertNotNull(result);
     assertTrue(result.shouldDie);
 
-    listener.unregister(attemptID, wid);
+    listener.unregister(attemptId, wid);
 
     // Verify after unregistration.
     result = listener.getTask(context);
     assertNotNull(result);
     assertTrue(result.shouldDie);
 
-    listener.stop();
-
     // test JVMID
     JVMId jvmid = JVMId.forName("jvm_001_002_m_004");
     assertNotNull(jvmid);
@@ -206,20 +254,10 @@ public class TestTaskAttemptListenerImpl {
     when(mockJob.getMapAttemptCompletionEvents(2, 100)).thenReturn(
         TypeConverter.fromYarn(empty));
 
-    AppContext appCtx = mock(AppContext.class);
+    configureMocks();
     when(appCtx.getJob(any(JobId.class))).thenReturn(mockJob);
-    JobTokenSecretManager secret = mock(JobTokenSecretManager.class);
-    RMHeartbeatHandler rmHeartbeatHandler =
-        mock(RMHeartbeatHandler.class);
-    final TaskHeartbeatHandler hbHandler = mock(TaskHeartbeatHandler.class);
-    Dispatcher dispatcher = mock(Dispatcher.class);
-    @SuppressWarnings("unchecked")
-    EventHandler<Event> ea = mock(EventHandler.class);
-    when(dispatcher.getEventHandler()).thenReturn(ea);
-    when(appCtx.getEventHandler()).thenReturn(ea);
-    CheckpointAMPreemptionPolicy policy = new CheckpointAMPreemptionPolicy();
-    policy.init(appCtx);
-    TaskAttemptListenerImpl listener = new MockTaskAttemptListenerImpl(
+
+    listener = new MockTaskAttemptListenerImpl(
         appCtx, secret, rmHeartbeatHandler, policy) {
       @Override
       protected void registerHeartbeatHandler(Configuration conf) {
@@ -262,26 +300,17 @@ public class TestTaskAttemptListenerImpl {
   public void testCommitWindow() throws IOException {
     SystemClock clock = SystemClock.getInstance();
 
+    configureMocks();
+
     org.apache.hadoop.mapreduce.v2.app.job.Task mockTask =
         mock(org.apache.hadoop.mapreduce.v2.app.job.Task.class);
     when(mockTask.canCommit(any(TaskAttemptId.class))).thenReturn(true);
     Job mockJob = mock(Job.class);
     when(mockJob.getTask(any(TaskId.class))).thenReturn(mockTask);
-    AppContext appCtx = mock(AppContext.class);
     when(appCtx.getJob(any(JobId.class))).thenReturn(mockJob);
     when(appCtx.getClock()).thenReturn(clock);
-    JobTokenSecretManager secret = mock(JobTokenSecretManager.class);
-    RMHeartbeatHandler rmHeartbeatHandler =
-        mock(RMHeartbeatHandler.class);
-    final TaskHeartbeatHandler hbHandler = mock(TaskHeartbeatHandler.class);
-    Dispatcher dispatcher = mock(Dispatcher.class);
-    @SuppressWarnings("unchecked")
-    EventHandler<Event> ea = mock(EventHandler.class);
-    when(dispatcher.getEventHandler()).thenReturn(ea);
-    when(appCtx.getEventHandler()).thenReturn(ea);
-    CheckpointAMPreemptionPolicy policy = new CheckpointAMPreemptionPolicy();
-    policy.init(appCtx);
-    TaskAttemptListenerImpl listener = new MockTaskAttemptListenerImpl(
+
+    listener = new MockTaskAttemptListenerImpl(
         appCtx, secret, rmHeartbeatHandler, policy) {
       @Override
       protected void registerHeartbeatHandler(Configuration conf) {
@@ -300,44 +329,29 @@ public class TestTaskAttemptListenerImpl {
     verify(mockTask, never()).canCommit(any(TaskAttemptId.class));
 
     // verify commit allowed when RM heartbeat is recent
-    when(rmHeartbeatHandler.getLastHeartbeatTime()).thenReturn(clock.getTime());
+    when(rmHeartbeatHandler.getLastHeartbeatTime())
+      .thenReturn(clock.getTime());
     canCommit = listener.canCommit(tid);
     assertTrue(canCommit);
     verify(mockTask, times(1)).canCommit(any(TaskAttemptId.class));
-
-    listener.stop();
   }
 
   @Test
   public void testCheckpointIDTracking()
     throws IOException, InterruptedException{
-
     SystemClock clock = SystemClock.getInstance();
 
+    configureMocks();
+
     org.apache.hadoop.mapreduce.v2.app.job.Task mockTask =
         mock(org.apache.hadoop.mapreduce.v2.app.job.Task.class);
     when(mockTask.canCommit(any(TaskAttemptId.class))).thenReturn(true);
     Job mockJob = mock(Job.class);
     when(mockJob.getTask(any(TaskId.class))).thenReturn(mockTask);
-
-    Dispatcher dispatcher = mock(Dispatcher.class);
-    @SuppressWarnings("unchecked")
-    EventHandler<Event> ea = mock(EventHandler.class);
-    when(dispatcher.getEventHandler()).thenReturn(ea);
-
-    RMHeartbeatHandler rmHeartbeatHandler =
-        mock(RMHeartbeatHandler.class);
-
-    AppContext appCtx = mock(AppContext.class);
     when(appCtx.getJob(any(JobId.class))).thenReturn(mockJob);
     when(appCtx.getClock()).thenReturn(clock);
-    when(appCtx.getEventHandler()).thenReturn(ea);
-    JobTokenSecretManager secret = mock(JobTokenSecretManager.class);
-    final TaskHeartbeatHandler hbHandler = mock(TaskHeartbeatHandler.class);
-    when(appCtx.getEventHandler()).thenReturn(ea);
-    CheckpointAMPreemptionPolicy policy = new CheckpointAMPreemptionPolicy();
-    policy.init(appCtx);
-    TaskAttemptListenerImpl listener = new MockTaskAttemptListenerImpl(
+
+    listener = new MockTaskAttemptListenerImpl(
         appCtx, secret, rmHeartbeatHandler, policy) {
       @Override
       protected void registerHeartbeatHandler(Configuration conf) {
@@ -387,42 +401,13 @@ public class TestTaskAttemptListenerImpl {
 
     //assert it worked
     assert outcid == incid;
-
-    listener.stop();
-
   }
 
-  @SuppressWarnings("rawtypes")
   @Test
   public void testStatusUpdateProgress()
       throws IOException, InterruptedException {
-    AppContext appCtx = mock(AppContext.class);
-    JobTokenSecretManager secret = mock(JobTokenSecretManager.class);
-    RMHeartbeatHandler rmHeartbeatHandler =
-        mock(RMHeartbeatHandler.class);
-    TaskHeartbeatHandler hbHandler = mock(TaskHeartbeatHandler.class);
-    Dispatcher dispatcher = mock(Dispatcher.class);
-    @SuppressWarnings("unchecked")
-    EventHandler<Event> ea = mock(EventHandler.class);
-    when(dispatcher.getEventHandler()).thenReturn(ea);
-
-    when(appCtx.getEventHandler()).thenReturn(ea);
-    CheckpointAMPreemptionPolicy policy = new CheckpointAMPreemptionPolicy();
-    policy.init(appCtx);
-    MockTaskAttemptListenerImpl listener =
-      new MockTaskAttemptListenerImpl(appCtx, secret,
-          rmHeartbeatHandler, hbHandler, policy);
-    Configuration conf = new Configuration();
-    listener.init(conf);
-    listener.start();
-    JVMId id = new JVMId("foo",1, true, 1);
-    WrappedJvmID wid = new WrappedJvmID(id.getJobId(), id.isMap, id.getId());
-
-    TaskAttemptID attemptID = new TaskAttemptID("1", 1, TaskType.MAP, 1, 1);
-    TaskAttemptId attemptId = TypeConverter.toYarn(attemptID);
-    Task task = mock(Task.class);
-    listener.registerPendingTask(task, wid);
-    listener.registerLaunchedTask(attemptId, wid);
+    configureMocks();
+    startListener(true);
     verify(hbHandler).register(attemptId);
 
     // make sure a ping doesn't report progress
@@ -437,6 +422,116 @@ public class TestTaskAttemptListenerImpl {
     feedback = listener.statusUpdate(attemptID, mockStatus);
     assertTrue(feedback.getTaskFound());
     verify(hbHandler).progressing(eq(attemptId));
-    listener.close();
+  }
+
+  @Test
+  public void testSingleStatusUpdate()
+      throws IOException, InterruptedException {
+    configureMocks();
+    startListener(true);
+
+    listener.statusUpdate(attemptID, firstReduceStatus);
+
+    verify(ea).handle(eventCaptor.capture());
+    TaskAttemptStatusUpdateEvent updateEvent =
+        (TaskAttemptStatusUpdateEvent) eventCaptor.getValue();
+
+    TaskAttemptStatus status = updateEvent.getTaskAttemptStatusRef().get();
+    assertTrue(status.fetchFailedMaps.contains(TASKATTEMPTID1));
+    assertEquals(1, status.fetchFailedMaps.size());
+    assertEquals(Phase.SHUFFLE, status.phase);
+  }
+
+  @Test
+  public void testStatusUpdateEventCoalescing()
+      throws IOException, InterruptedException {
+    configureMocks();
+    startListener(true);
+
+    listener.statusUpdate(attemptID, firstReduceStatus);
+    listener.statusUpdate(attemptID, secondReduceStatus);
+
+    verify(ea).handle(any(Event.class));
+    ConcurrentMap<TaskAttemptId,
+        AtomicReference<TaskAttemptStatus>> attemptIdToStatus =
+        listener.getAttemptIdToStatus();
+    TaskAttemptStatus status = attemptIdToStatus.get(attemptId).get();
+
+    assertTrue(status.fetchFailedMaps.contains(TASKATTEMPTID1));
+    assertTrue(status.fetchFailedMaps.contains(TASKATTEMPTID2));
+    assertEquals(2, status.fetchFailedMaps.size());
+    assertEquals(Phase.SORT, status.phase);
+  }
+
+  @Test
+  public void testCoalescedStatusUpdatesCleared()
+      throws IOException, InterruptedException {
+    // First two events are coalesced, the third is not
+    configureMocks();
+    startListener(true);
+
+    listener.statusUpdate(attemptID, firstReduceStatus);
+    listener.statusUpdate(attemptID, secondReduceStatus);
+    ConcurrentMap<TaskAttemptId,
+        AtomicReference<TaskAttemptStatus>> attemptIdToStatus =
+        listener.getAttemptIdToStatus();
+    attemptIdToStatus.get(attemptId).set(null);
+    listener.statusUpdate(attemptID, thirdReduceStatus);
+
+    verify(ea, times(2)).handle(eventCaptor.capture());
+    TaskAttemptStatusUpdateEvent updateEvent =
+        (TaskAttemptStatusUpdateEvent) eventCaptor.getValue();
+
+    TaskAttemptStatus status = updateEvent.getTaskAttemptStatusRef().get();
+    assertNull(status.fetchFailedMaps);
+    assertEquals(Phase.REDUCE, status.phase);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testStatusUpdateFromUnregisteredTask()
+      throws IOException, InterruptedException{
+    configureMocks();
+    startListener(false);
+
+    listener.statusUpdate(attemptID, firstReduceStatus);
+  }
+
+  private void configureMocks() {
+    firstReduceStatus = new ReduceTaskStatus(attemptID, 0.0f, 1,
+        TaskStatus.State.RUNNING, "", "RUNNING", "", TaskStatus.Phase.SHUFFLE,
+        new Counters());
+    firstReduceStatus.addFetchFailedMap(TaskAttemptID.forName(ATTEMPT1_ID));
+
+    secondReduceStatus = new ReduceTaskStatus(attemptID, 0.0f, 1,
+        TaskStatus.State.RUNNING, "", "RUNNING", "", TaskStatus.Phase.SORT,
+        new Counters());
+    secondReduceStatus.addFetchFailedMap(TaskAttemptID.forName(ATTEMPT2_ID));
+
+    thirdReduceStatus = new ReduceTaskStatus(attemptID, 0.0f, 1,
+        TaskStatus.State.RUNNING, "", "RUNNING", "",
+        TaskStatus.Phase.REDUCE, new Counters());
+
+    when(dispatcher.getEventHandler()).thenReturn(ea);
+    when(appCtx.getEventHandler()).thenReturn(ea);
+    policy = new CheckpointAMPreemptionPolicy();
+    policy.init(appCtx);
+    listener = new MockTaskAttemptListenerImpl(appCtx, secret,
+          rmHeartbeatHandler, hbHandler, policy);
+    id = new JVMId("foo", 1, true, 1);
+    wid = new WrappedJvmID(id.getJobId(), id.isMap, id.getId());
+    attemptID = new TaskAttemptID("1", 1, TaskType.MAP, 1, 1);
+    attemptId = TypeConverter.toYarn(attemptID);
+  }
+
+  private void startListener(boolean registerTask) {
+    Configuration conf = new Configuration();
+
+    listener.init(conf);
+    listener.start();
+
+    if (registerTask) {
+      listener.registerPendingTask(task, wid);
+      listener.registerLaunchedTask(attemptId, wid);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21d36273/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java
index cb2a29e..67a8901 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertEquals;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicReference;
 
 import com.google.common.base.Supplier;
 import org.apache.hadoop.conf.Configuration;
@@ -442,7 +443,7 @@ public class TestFetchFailure {
     status.stateString = "OK";
     status.taskState = attempt.getState();
     TaskAttemptStatusUpdateEvent event = new TaskAttemptStatusUpdateEvent(attempt.getID(),
-        status);
+        new AtomicReference<>(status));
     app.getContext().getEventHandler().handle(event);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21d36273/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java
index 77f9a09..ca3c28c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java
@@ -24,6 +24,7 @@ import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Iterator;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.junit.Assert;
 
@@ -103,7 +104,8 @@ public class TestMRClientService {
     taskAttemptStatus.phase = Phase.MAP;
     // send the status update
     app.getContext().getEventHandler().handle(
-        new TaskAttemptStatusUpdateEvent(attempt.getID(), taskAttemptStatus));
+        new TaskAttemptStatusUpdateEvent(attempt.getID(),
+            new AtomicReference<>(taskAttemptStatus)));
 
     
     //verify that all object are fully populated by invoking RPCs.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/21d36273/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecutionWithMRApp.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecutionWithMRApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecutionWithMRApp.java
index e8003c0..de171c7 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecutionWithMRApp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestSpeculativeExecutionWithMRApp.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Random;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.junit.Assert;
 import org.apache.hadoop.conf.Configuration;
@@ -84,7 +85,8 @@ public class TestSpeculativeExecutionWithMRApp {
             createTaskAttemptStatus(taskAttempt.getKey(), (float) 0.8,
               TaskAttemptState.RUNNING);
         TaskAttemptStatusUpdateEvent event =
-            new TaskAttemptStatusUpdateEvent(taskAttempt.getKey(), status);
+            new TaskAttemptStatusUpdateEvent(taskAttempt.getKey(),
+                new AtomicReference<>(status));
         appEventHandler.handle(event);
       }
     }
@@ -155,7 +157,8 @@ public class TestSpeculativeExecutionWithMRApp {
             createTaskAttemptStatus(taskAttempt.getKey(), (float) 0.5,
               TaskAttemptState.RUNNING);
         TaskAttemptStatusUpdateEvent event =
-            new TaskAttemptStatusUpdateEvent(taskAttempt.getKey(), status);
+            new TaskAttemptStatusUpdateEvent(taskAttempt.getKey(),
+                new AtomicReference<>(status));
         appEventHandler.handle(event);
       }
     }
@@ -180,7 +183,8 @@ public class TestSpeculativeExecutionWithMRApp {
                 TaskAttemptState.RUNNING);
           speculatedTask = task.getValue();
           TaskAttemptStatusUpdateEvent event =
-              new TaskAttemptStatusUpdateEvent(taskAttempt.getKey(), status);
+              new TaskAttemptStatusUpdateEvent(taskAttempt.getKey(),
+                  new AtomicReference<>(status));
           appEventHandler.handle(event);
         }
       }
@@ -195,7 +199,8 @@ public class TestSpeculativeExecutionWithMRApp {
               createTaskAttemptStatus(taskAttempt.getKey(), (float) 0.75,
                 TaskAttemptState.RUNNING);
           TaskAttemptStatusUpdateEvent event =
-              new TaskAttemptStatusUpdateEvent(taskAttempt.getKey(), status);
+              new TaskAttemptStatusUpdateEvent(taskAttempt.getKey(),
+                  new AtomicReference<>(status));
           appEventHandler.handle(event);
         }
       }


---------------------------------------------------------------------
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-12872. EC Checksum broken when BlockAccessToken is enabled.

Posted by ae...@apache.org.
HDFS-12872. EC Checksum broken when BlockAccessToken is enabled.


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

Branch: refs/heads/HDFS-7240
Commit: 56b1ff80dd9fbcde8d21a604eff0babb3a16418f
Parents: 05c347f
Author: Xiao Chen <xi...@apache.org>
Authored: Tue Dec 5 20:48:02 2017 -0800
Committer: Xiao Chen <xi...@apache.org>
Committed: Tue Dec 5 20:48:16 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java | 3 ++-
 .../hadoop/hdfs/server/blockmanagement/BlockManager.java    | 9 ++++-----
 .../test/java/org/apache/hadoop/hdfs/TestFileChecksum.java  | 9 ++++++++-
 3 files changed, 14 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/56b1ff80/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
index 4dbaa01..e7d5b3a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
@@ -192,6 +192,7 @@ public class LocatedBlocks {
         + "\n  underConstruction=" + underConstruction
         + "\n  blocks=" + blocks
         + "\n  lastLocatedBlock=" + lastLocatedBlock
-        + "\n  isLastBlockComplete=" + isLastBlockComplete + "}";
+        + "\n  isLastBlockComplete=" + isLastBlockComplete
+        + "\n  ecPolicy=" + ecPolicy + "}";
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56b1ff80/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 1cdb159..ae04aac 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -1378,12 +1378,11 @@ public class BlockManager implements BlockStatsMXBean {
               b.getStorageIDs());
         }
         sb.setBlockTokens(blockTokens);
-      } else {
-        b.setBlockToken(blockTokenSecretManager.generateToken(
-            NameNode.getRemoteUser().getShortUserName(),
-            b.getBlock(), EnumSet.of(mode), b.getStorageTypes(),
-            b.getStorageIDs()));
       }
+      b.setBlockToken(blockTokenSecretManager.generateToken(
+          NameNode.getRemoteUser().getShortUserName(),
+          b.getBlock(), EnumSet.of(mode), b.getStorageTypes(),
+          b.getStorageIDs()));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56b1ff80/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java
index 9d6687c..d201ce1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileChecksum.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -33,8 +34,12 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.slf4j.event.Level;
 
 import java.io.IOException;
+import java.util.Random;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY;
 
 /**
  * This test serves a prototype to demo the idea proposed so far. It creates two
@@ -77,6 +82,7 @@ public class TestFileChecksum {
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
         false);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
+    conf.setBoolean(DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
     Path ecPath = new Path(ecDir);
     cluster.getFileSystem().mkdir(ecPath, FsPermission.getDirDefault());
@@ -89,6 +95,7 @@ public class TestFileChecksum {
     bytesPerCRC = conf.getInt(
         HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY,
         HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT);
+    GenericTestUtils.setLogLevel(FileChecksumHelper.LOG, Level.DEBUG);
   }
 
   @After
@@ -518,7 +525,7 @@ public class TestFileChecksum {
 
     LocatedBlock locatedBlock = locatedBlocks.get(0);
     DatanodeInfo[] datanodes = locatedBlock.getLocations();
-    DatanodeInfo chosenDn = datanodes[0];
+    DatanodeInfo chosenDn = datanodes[new Random().nextInt(datanodes.length)];
 
     int idx = 0;
     for (DataNode dn : cluster.getDataNodes()) {


---------------------------------------------------------------------
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: HADOOP-14600. LocatedFileStatus constructor forces RawLocalFS to exec a process to get the permissions. Contributed by Ping Liu

Posted by ae...@apache.org.
HADOOP-14600. LocatedFileStatus constructor forces RawLocalFS to exec a process to get the permissions. Contributed by Ping Liu


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

Branch: refs/heads/HDFS-7240
Commit: f9d195dfe9cc2c3e4659c3475319ac7c937b5c44
Parents: 60f95fb
Author: Chris Douglas <cd...@apache.org>
Authored: Fri Dec 1 11:41:38 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 1 22:12:26 2017 -0800

----------------------------------------------------------------------
 .../apache/hadoop/fs/RawLocalFileSystem.java    |  77 +++++++++-
 .../hadoop/fs/permission/FsPermission.java      |  34 +++++
 .../org/apache/hadoop/io/nativeio/NativeIO.java |  34 +++++
 .../org/apache/hadoop/io/nativeio/NativeIO.c    |  85 +++++++++++
 .../fs/TestRawLocalFileSystemContract.java      |  87 ++++++++++-
 .../hadoop/fs/permission/TestFsPermission.java  |  31 ++++
 .../apache/hadoop/io/nativeio/TestNativeIO.java | 144 ++++++++++++++++---
 .../java/org/apache/hadoop/test/StatUtils.java  | 126 ++++++++++++++++
 8 files changed, 594 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9d195df/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
index 721d0c0..c0f8199 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
@@ -696,11 +696,34 @@ public class RawLocalFileSystem extends FileSystem {
       return super.getGroup();
     }
 
+    /**
+     * Load file permission information (UNIX symbol rwxrwxrwx, sticky bit info).
+     *
+     * To improve peformance, give priority to native stat() call. First try get
+     * permission information by using native JNI call then fall back to use non
+     * native (ProcessBuilder) call in case native lib is not loaded or native
+     * call is not successful
+     */
+    private synchronized void loadPermissionInfo() {
+      if (!isPermissionLoaded() && NativeIO.isAvailable()) {
+        try {
+          loadPermissionInfoByNativeIO();
+        } catch (IOException ex) {
+          LOG.debug("Native call failed", ex);
+        }
+      }
+
+      if (!isPermissionLoaded()) {
+        loadPermissionInfoByNonNativeIO();
+      }
+    }
+
     /// loads permissions, owner, and group from `ls -ld`
-    private void loadPermissionInfo() {
+    @VisibleForTesting
+    void loadPermissionInfoByNonNativeIO() {
       IOException e = null;
       try {
-        String output = FileUtil.execCommand(new File(getPath().toUri()), 
+        String output = FileUtil.execCommand(new File(getPath().toUri()),
             Shell.getGetPermissionCommand());
         StringTokenizer t =
             new StringTokenizer(output, Shell.TOKEN_SEPARATOR_REGEX);
@@ -716,16 +739,16 @@ public class RawLocalFileSystem extends FileSystem {
         t.nextToken();
 
         String owner = t.nextToken();
+        String group = t.nextToken();
         // If on windows domain, token format is DOMAIN\\user and we want to
         // extract only the user name
+        // same as to the group name
         if (Shell.WINDOWS) {
-          int i = owner.indexOf('\\');
-          if (i != -1)
-            owner = owner.substring(i + 1);
+          owner = removeDomain(owner);
+          group = removeDomain(group);
         }
         setOwner(owner);
-
-        setGroup(t.nextToken());
+        setGroup(group);
       } catch (Shell.ExitCodeException ioe) {
         if (ioe.getExitCode() != 1) {
           e = ioe;
@@ -745,6 +768,46 @@ public class RawLocalFileSystem extends FileSystem {
       }
     }
 
+    // In Windows, domain name is added.
+    // For example, given machine name (domain name) dname, user name i, then
+    // the result for user is dname\\i and for group is dname\\None. So we need
+    // remove domain name as follows:
+    // DOMAIN\\user => user, DOMAIN\\group => group
+    private String removeDomain(String str) {
+      int index = str.indexOf("\\");
+      if (index != -1) {
+        str = str.substring(index + 1);
+      }
+      return str;
+    }
+
+    // loads permissions, owner, and group from `ls -ld`
+    // but use JNI to more efficiently get file mode (permission, owner, group)
+    // by calling file stat() in *nix or some similar calls in Windows
+    @VisibleForTesting
+    void loadPermissionInfoByNativeIO() throws IOException {
+      Path path = getPath();
+      String pathName = path.toUri().getPath();
+      // remove leading slash for Windows path
+      if (Shell.WINDOWS && pathName.startsWith("/")) {
+        pathName = pathName.substring(1);
+      }
+      try {
+        NativeIO.POSIX.Stat stat = NativeIO.POSIX.getStat(pathName);
+        String owner = stat.getOwner();
+        String group = stat.getGroup();
+        int mode = stat.getMode();
+        setOwner(owner);
+        setGroup(group);
+        setPermission(new FsPermission(mode));
+      } catch (IOException e) {
+        setOwner(null);
+        setGroup(null);
+        setPermission(null);
+        throw e;
+      }
+    }
+
     @Override
     public void write(DataOutput out) throws IOException {
       if (!isPermissionLoaded()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9d195df/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
index 031092b..51c113a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
@@ -91,6 +91,40 @@ public class FsPermission implements Writable, Serializable,
   public FsPermission(short mode) { fromShort(mode); }
 
   /**
+   * Construct by the given mode.
+   *
+   * octal mask is applied.
+   *
+   *<pre>
+   *              before mask     after mask    file type   sticky bit
+   *
+   *    octal     100644            644         file          no
+   *    decimal    33188            420
+   *
+   *    octal     101644           1644         file          yes
+   *    decimal    33700           1420
+   *
+   *    octal      40644            644         directory     no
+   *    decimal    16804            420
+   *
+   *    octal      41644           1644         directory     yes
+   *    decimal    17316           1420
+   *</pre>
+   *
+   * 100644 becomes 644 while 644 remains as 644
+   *
+   * @param mode Mode is supposed to come from the result of native stat() call.
+   *             It contains complete permission information: rwxrwxrwx, sticky
+   *             bit, whether it is a directory or a file, etc. Upon applying
+   *             mask, only permission and sticky bit info will be kept because
+   *             they are the only parts to be used for now.
+   * @see #FsPermission(short mode)
+   */
+  public FsPermission(int mode) {
+    this((short)(mode & 01777));
+  }
+
+  /**
    * Copy constructor
    * 
    * @param other other permission

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9d195df/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
index 84cd42c..f601edd 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.HardLink;
+import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.SecureIOUtils.AlreadyExistsException;
 import org.apache.hadoop.util.NativeCodeLoader;
@@ -221,6 +222,8 @@ public class NativeIO {
     public static native FileDescriptor open(String path, int flags, int mode) throws IOException;
     /** Wrapper around fstat(2) */
     private static native Stat fstat(FileDescriptor fd) throws IOException;
+    /** Wrapper around stat(2). */
+    private static native Stat stat(String path) throws IOException;
 
     /** Native chmod implementation. On UNIX, it is a wrapper around chmod(2) */
     private static native void chmodImpl(String path, int mode) throws IOException;
@@ -428,6 +431,37 @@ public class NativeIO {
       return stat;
     }
 
+    /**
+     * Return the file stat for a file path.
+     *
+     * @param path  file path
+     * @return  the file stat
+     * @throws IOException  thrown if there is an IO error while obtaining the
+     * file stat
+     */
+    public static Stat getStat(String path) throws IOException {
+      if (path == null) {
+        String errMessage = "Path is null";
+        LOG.warn(errMessage);
+        throw new IOException(errMessage);
+      }
+      Stat stat = null;
+      try {
+        if (!Shell.WINDOWS) {
+          stat = stat(path);
+          stat.owner = getName(IdCache.USER, stat.ownerId);
+          stat.group = getName(IdCache.GROUP, stat.groupId);
+        } else {
+          stat = stat(path);
+        }
+      } catch (NativeIOException nioe) {
+        LOG.warn("NativeIO.getStat error ({}): {} -- file path: {}",
+            nioe.getErrorCode(), nioe.getMessage(), path);
+        throw new PathIOException(path, nioe);
+      }
+      return stat;
+    }
+
     private static String getName(IdCache domain, int id) throws IOException {
       Map<Integer, CachedName> idNameCache = (domain == IdCache.USER)
         ? USER_ID_NAME_CACHE : GROUP_ID_NAME_CACHE;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9d195df/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
index 242a456..2274d57 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
@@ -383,7 +383,92 @@ cleanup:
 #endif
 }
 
+/*
+ * Class:     org_apache_hadoop_io_nativeio_NativeIO_POSIX
+ * Method:    stat
+ * Signature: (Ljava/lang/String;)Lorg/apache/hadoop/io/nativeio/NativeIO$POSIX$Stat;
+ * public static native Stat stat(String path);
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT jobject JNICALL
+Java_org_apache_hadoop_io_nativeio_NativeIO_00024POSIX_stat(
+  JNIEnv *env, jclass clazz, jstring j_path)
+{
+#ifdef UNIX
+  jobject ret = NULL;
+
+  const char *c_path = (*env)->GetStringUTFChars(env, j_path, NULL);
+  if (c_path == NULL) {
+    goto cleanup;
+  }
+
+  struct stat s;
+  int rc = stat(c_path, &s);
+  if (rc != 0) {
+    throw_ioe(env, errno);
+    goto cleanup;
+  }
+
+  // Construct result
+  ret = (*env)->NewObject(env, stat_clazz, stat_ctor,
+    (jint)s.st_uid, (jint)s.st_gid, (jint)s.st_mode);
+
+cleanup:
+  if (c_path != NULL) {
+    (*env)->ReleaseStringUTFChars(env, j_path, c_path);
+  }
+  return ret;
+#endif
 
+#ifdef WINDOWS
+  LPWSTR owner = NULL;
+  LPWSTR group = NULL;
+  int mode = 0;
+  jstring jstr_owner = NULL;
+  jstring jstr_group = NULL;
+  int rc;
+  jobject ret = NULL;
+
+  LPCWSTR path = (LPCWSTR) (*env)->GetStringChars(env, j_path, NULL);
+  if (path == NULL) {
+    goto cleanup;
+  }
+
+  rc = FindFileOwnerAndPermission(path, TRUE, &owner, &group, &mode);
+  if (rc != ERROR_SUCCESS) {
+    throw_ioe(env, rc);
+    goto cleanup;
+  }
+
+  jstr_owner = (*env)->NewString(env, owner, (jsize) wcslen(owner));
+  if (jstr_owner == NULL) goto cleanup;
+
+  jstr_group = (*env)->NewString(env, group, (jsize) wcslen(group));
+  if (jstr_group == NULL) goto cleanup;
+
+  ret = (*env)->NewObject(env, stat_clazz, stat_ctor2,
+    jstr_owner, jstr_group, (jint)mode);
+
+cleanup:
+  if (path != NULL)
+    (*env)->ReleaseStringChars(env, j_path, (const jchar*) path);
+
+  if (ret == NULL) {
+    if (jstr_owner != NULL)
+      (*env)->ReleaseStringChars(env, jstr_owner, owner);
+
+    if (jstr_group != NULL)
+      (*env)->ReleaseStringChars(env, jstr_group, group);
+  }
+
+  LocalFree(owner);
+  LocalFree(group);
+
+  return ret;
+#endif
+}
 
 /**
  * public static native void posix_fadvise(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9d195df/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestRawLocalFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestRawLocalFileSystemContract.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestRawLocalFileSystemContract.java
index b023c09..ebf9ea7 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestRawLocalFileSystemContract.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestRawLocalFileSystemContract.java
@@ -17,10 +17,18 @@
  */
 package org.apache.hadoop.fs;
 
+import java.io.File;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.StatUtils;
 import org.apache.hadoop.util.Shell;
+
 import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -78,4 +86,81 @@ public class TestRawLocalFileSystemContract extends FileSystemContractBaseTest {
   protected boolean filesystemIsCaseSensitive() {
     return !(Shell.WINDOWS || Shell.MAC);
   }
-}
\ No newline at end of file
+
+  // cross-check getPermission using both native/non-native
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testPermission() throws Exception {
+    Path testDir = getTestBaseDir();
+    String testFilename = "teststat2File";
+    Path path = new Path(testDir, testFilename);
+
+    RawLocalFileSystem rfs = new RawLocalFileSystem();
+    Configuration conf = new Configuration();
+    rfs.initialize(rfs.getUri(), conf);
+    rfs.createNewFile(path);
+
+    File file = rfs.pathToFile(path);
+    long defaultBlockSize = rfs.getDefaultBlockSize(path);
+
+    //
+    // test initial permission
+    //
+    RawLocalFileSystem.DeprecatedRawLocalFileStatus fsNIO =
+      new RawLocalFileSystem.DeprecatedRawLocalFileStatus(
+          file, defaultBlockSize, rfs);
+    fsNIO.loadPermissionInfoByNativeIO();
+    RawLocalFileSystem.DeprecatedRawLocalFileStatus fsnonNIO =
+        new RawLocalFileSystem.DeprecatedRawLocalFileStatus(
+            file, defaultBlockSize, rfs);
+    fsnonNIO.loadPermissionInfoByNonNativeIO();
+
+    assertEquals(fsNIO.getOwner(), fsnonNIO.getOwner());
+    assertEquals(fsNIO.getGroup(), fsnonNIO.getGroup());
+    assertEquals(fsNIO.getPermission(), fsnonNIO.getPermission());
+
+    LOG.info("owner: {}, group: {}, permission: {}, isSticky: {}",
+        fsNIO.getOwner(), fsNIO.getGroup(), fsNIO.getPermission(),
+        fsNIO.getPermission().getStickyBit());
+
+    //
+    // test normal chmod - no sticky bit
+    //
+    StatUtils.setPermissionFromProcess("644", file.getPath());
+    fsNIO.loadPermissionInfoByNativeIO();
+    fsnonNIO.loadPermissionInfoByNonNativeIO();
+    assertEquals(fsNIO.getPermission(), fsnonNIO.getPermission());
+    assertEquals(644, fsNIO.getPermission().toOctal());
+    assertFalse(fsNIO.getPermission().getStickyBit());
+    assertFalse(fsnonNIO.getPermission().getStickyBit());
+
+    //
+    // test sticky bit
+    // unfortunately, cannot be done in Windows environments
+    //
+    if (!Shell.WINDOWS) {
+      //
+      // add sticky bit
+      //
+      StatUtils.setPermissionFromProcess("1644", file.getPath());
+      fsNIO.loadPermissionInfoByNativeIO();
+      fsnonNIO.loadPermissionInfoByNonNativeIO();
+      assertEquals(fsNIO.getPermission(), fsnonNIO.getPermission());
+      assertEquals(1644, fsNIO.getPermission().toOctal());
+      assertEquals(true, fsNIO.getPermission().getStickyBit());
+      assertEquals(true, fsnonNIO.getPermission().getStickyBit());
+
+      //
+      // remove sticky bit
+      //
+      StatUtils.setPermissionFromProcess("-t", file.getPath());
+      fsNIO.loadPermissionInfoByNativeIO();
+      fsnonNIO.loadPermissionInfoByNonNativeIO();
+      assertEquals(fsNIO.getPermission(), fsnonNIO.getPermission());
+      assertEquals(644, fsNIO.getPermission().toOctal());
+      assertEquals(false, fsNIO.getPermission().getStickyBit());
+      assertEquals(false, fsnonNIO.getPermission().getStickyBit());
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9d195df/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/permission/TestFsPermission.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/permission/TestFsPermission.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/permission/TestFsPermission.java
index afddf80..0c5b415 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/permission/TestFsPermission.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/permission/TestFsPermission.java
@@ -264,6 +264,37 @@ public class TestFsPermission {
            msg.contains("octal or symbolic");
   }
 
+  /**
+   *  test FsPermission(int) constructor.
+   */
+  @Test
+  public void testIntPermission() {
+    // Octal           Decimals        Masked OCT      Masked DEC
+    // 100644          33188           644             420
+    // 101644          33700           1644            932
+    // 40644           16804           644             420
+    // 41644           17316           1644            932
+    // 644             420             644             420
+    // 1644            932             1644            932
+
+    int[][] permission_mask_maps = {
+      // Octal                 Decimal    Unix Symbolic
+      { 0100644,  0644, 0 },   // 33188    -rw-r--
+      { 0101644, 01644, 1 },   // 33700    -rw-r-t
+      { 040644,   0644, 0 },   // 16804    drw-r--
+      { 041644,  01644, 1 }    // 17316    drw-r-t
+    };
+
+    for (int[] permission_mask_map : permission_mask_maps) {
+      int original_permission_value = permission_mask_map[0];
+      int masked_permission_value = permission_mask_map[1];
+      boolean hasStickyBit = permission_mask_map[2] == 1;
+      FsPermission fsPermission = new FsPermission(original_permission_value);
+      assertEquals(masked_permission_value, fsPermission.toShort());
+      assertEquals(hasStickyBit, fsPermission.getStickyBit());
+    }
+  }
+
   // Symbolic umask list is generated in linux shell using by the command:
   // umask 0; umask <octal number>; umask -S
   static final String[][] SYMBOLIC = new String[][] {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9d195df/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java
index 6989905..6b3c232 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java
@@ -29,36 +29,44 @@ import java.nio.MappedByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.channels.FileChannel.MapMode;
 import java.util.Random;
-import java.util.concurrent.atomic.AtomicReference;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assume.*;
-import static org.junit.Assert.*;
-import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows;
-import static org.apache.hadoop.test.PlatformAssumptions.assumeWindows;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.commons.io.FileUtils;
+
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.test.StatUtils;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.apache.hadoop.util.Time;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.*;
 import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.Stat.*;
+import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows;
+import static org.apache.hadoop.test.PlatformAssumptions.assumeWindows;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assume.*;
+import static org.junit.Assert.*;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class TestNativeIO {
   static final Logger LOG = LoggerFactory.getLogger(TestNativeIO.class);
@@ -164,6 +172,110 @@ public class TestNativeIO {
   }
 
   @Test (timeout = 30000)
+  public void testStat() throws Exception {
+    Configuration conf = new Configuration();
+    FileSystem fileSystem = FileSystem.getLocal(conf).getRawFileSystem();
+    Path path = new Path(TEST_DIR.getPath(), "teststat2");
+    fileSystem.createNewFile(path);
+    String testFilePath = path.toString();
+
+    try {
+      doStatTest(testFilePath);
+      LOG.info("testStat() is successful.");
+    } finally {
+      ContractTestUtils.cleanup("cleanup test file: " + path.toString(),
+          fileSystem, path);
+    }
+  }
+
+  private boolean doStatTest(String testFilePath) throws Exception {
+    NativeIO.POSIX.Stat stat = NativeIO.POSIX.getStat(testFilePath);
+    String owner = stat.getOwner();
+    String group = stat.getGroup();
+    int mode = stat.getMode();
+
+    // direct check with System
+    String expectedOwner = System.getProperty("user.name");
+    assertEquals(expectedOwner, owner);
+    assertNotNull(group);
+    assertTrue(!group.isEmpty());
+
+    // cross check with ProcessBuilder
+    StatUtils.Permission expected =
+        StatUtils.getPermissionFromProcess(testFilePath);
+    StatUtils.Permission permission =
+        new StatUtils.Permission(owner, group, new FsPermission(mode));
+
+    assertEquals(expected.getOwner(), permission.getOwner());
+    assertEquals(expected.getGroup(), permission.getGroup());
+    assertEquals(expected.getFsPermission(), permission.getFsPermission());
+
+    LOG.info("Load permission test is successful for path: {}, stat: {}",
+        testFilePath, stat);
+    LOG.info("On mask, stat is owner: {}, group: {}, permission: {}",
+        owner, group, permission.getFsPermission().toOctal());
+    return true;
+  }
+
+  @Test
+  public void testStatOnError() throws Exception {
+    final String testNullFilePath = null;
+    LambdaTestUtils.intercept(IOException.class,
+            "Path is null",
+            () -> NativeIO.POSIX.getStat(testNullFilePath));
+
+    final String testInvalidFilePath = "C:\\nonexisting_path\\nonexisting_file";
+    LambdaTestUtils.intercept(IOException.class,
+            PathIOException.class.getName(),
+            () -> NativeIO.POSIX.getStat(testInvalidFilePath));
+  }
+
+  @Test (timeout = 30000)
+  public void testMultiThreadedStat() throws Exception {
+    Configuration conf = new Configuration();
+    FileSystem fileSystem = FileSystem.getLocal(conf).getRawFileSystem();
+    Path path = new Path(TEST_DIR.getPath(), "teststat2");
+    fileSystem.createNewFile(path);
+    String testFilePath = path.toString();
+
+    int numOfThreads = 10;
+    ExecutorService executorService =
+        Executors.newFixedThreadPool(numOfThreads);
+    executorService.awaitTermination(1000, TimeUnit.MILLISECONDS);
+    try {
+      for (int i = 0; i < numOfThreads; i++){
+        Future<Boolean> result =
+            executorService.submit(() -> doStatTest(testFilePath));
+        assertTrue(result.get());
+      }
+      LOG.info("testMultiThreadedStat() is successful.");
+    } finally {
+      executorService.shutdown();
+      ContractTestUtils.cleanup("cleanup test file: " + path.toString(),
+          fileSystem, path);
+    }
+  }
+
+  @Test
+  public void testMultiThreadedStatOnError() throws Exception {
+    final String testInvalidFilePath = "C:\\nonexisting_path\\nonexisting_file";
+
+    int numOfThreads = 10;
+    ExecutorService executorService =
+        Executors.newFixedThreadPool(numOfThreads);
+    for (int i = 0; i < numOfThreads; i++) {
+      try {
+        Future<Boolean> result =
+            executorService.submit(() -> doStatTest(testInvalidFilePath));
+        result.get();
+      } catch (Exception e) {
+        assertTrue(e.getCause() instanceof PathIOException);
+      }
+    }
+    executorService.shutdown();
+  }
+
+  @Test (timeout = 30000)
   public void testSetFilePointer() throws Exception {
     assumeWindows();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9d195df/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/StatUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/StatUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/StatUtils.java
new file mode 100644
index 0000000..fef35d0
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/StatUtils.java
@@ -0,0 +1,126 @@
+/**
+ * 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.test;
+
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Shell;
+
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.StringTokenizer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Helper class for stat/permission utility methods. Forks processes to query
+ * permission info.
+ */
+public class StatUtils {
+  public static class Permission {
+    private String owner;
+    private String group;
+    private FsPermission fsPermission;
+
+    public Permission(String owner, String group, FsPermission fsPermission) {
+      this.owner = owner;
+      this.group = group;
+      this.fsPermission = fsPermission;
+    }
+
+    public String getOwner() {
+      return owner;
+    }
+
+    public String getGroup() {
+      return group;
+    }
+
+    public FsPermission getFsPermission() {
+      return fsPermission;
+    }
+  }
+
+  public static Permission getPermissionFromProcess(String filePath)
+      throws Exception {
+    String[] shellCommand = Shell.getGetPermissionCommand();
+    String sPerm = getPermissionStringFromProcess(shellCommand, filePath);
+
+    StringTokenizer tokenizer =
+        new StringTokenizer(sPerm, Shell.TOKEN_SEPARATOR_REGEX);
+    String symbolicPermission = tokenizer.nextToken();
+    tokenizer.nextToken(); // skip hard link
+    String owner = tokenizer.nextToken();
+    String group = tokenizer.nextToken();
+    if (Shell.WINDOWS) {
+      owner = removeDomain(owner);
+      group = removeDomain(group);
+    }
+
+    Permission permission =
+        new Permission(owner, group, FsPermission.valueOf(symbolicPermission));
+
+    return permission;
+  }
+
+  public static void setPermissionFromProcess(String chmod, String filePath)
+      throws Exception {
+    setPermissionFromProcess(chmod, false, filePath);
+  }
+
+  public static void setPermissionFromProcess(String chmod, boolean recursive,
+      String filePath) throws Exception {
+    String[] shellCommand = Shell.getSetPermissionCommand(chmod, recursive);
+    getPermissionStringFromProcess(shellCommand, filePath);
+  }
+
+  private static String removeDomain(String str) {
+    int index = str.indexOf("\\");
+    if (index != -1) {
+      str = str.substring(index + 1);
+    }
+    return str;
+  }
+
+  private static String getPermissionStringFromProcess(String[] shellCommand,
+      String testFilePath) throws Exception {
+    List<String> cmd = new ArrayList(Arrays.asList(shellCommand));
+    cmd.add(testFilePath);
+
+    ProcessBuilder processBuilder = new ProcessBuilder(cmd);
+    Process process = processBuilder.start();
+
+    ExecutorService executorService = Executors.newSingleThreadExecutor();
+    executorService.awaitTermination(2000, TimeUnit.MILLISECONDS);
+    try {
+      Future<String> future =
+          executorService.submit(() -> new BufferedReader(
+              new InputStreamReader(process.getInputStream(),
+                  Charset.defaultCharset())).lines().findFirst().orElse(""));
+      return future.get();
+    } finally {
+      process.destroy();
+      executorService.shutdown();
+    }
+  }
+}


---------------------------------------------------------------------
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: HDFS-11576. Block recovery will fail indefinitely if recovery time > heartbeat interval. Contributed by Lukas Majercak

Posted by ae...@apache.org.
HDFS-11576. Block recovery will fail indefinitely if recovery time > heartbeat interval. Contributed by Lukas Majercak


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

Branch: refs/heads/HDFS-7240
Commit: 42307e3c3abbfe0b83d9a2581deba327435b910f
Parents: f9d195d
Author: Chris Douglas <cd...@apache.org>
Authored: Fri Dec 1 22:34:30 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 1 22:34:30 2017 -0800

----------------------------------------------------------------------
 .../apache/hadoop/test/GenericTestUtils.java    |  10 +-
 .../server/blockmanagement/BlockManager.java    |  40 ++++++
 .../blockmanagement/PendingRecoveryBlocks.java  | 143 +++++++++++++++++++
 .../hdfs/server/namenode/FSNamesystem.java      |  40 +++---
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |  10 ++
 .../TestPendingRecoveryBlocks.java              |  87 +++++++++++
 .../hdfs/server/datanode/TestBlockRecovery.java | 107 ++++++++++++++
 .../namenode/ha/TestPipelinesFailover.java      |   4 +-
 8 files changed, 421 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/42307e3c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
index 0db6c73..cdde48c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
@@ -641,10 +641,16 @@ public abstract class GenericTestUtils {
    * conditions.
    */
   public static class SleepAnswer implements Answer<Object> {
+    private final int minSleepTime;
     private final int maxSleepTime;
     private static Random r = new Random();
-    
+
     public SleepAnswer(int maxSleepTime) {
+      this(0, maxSleepTime);
+    }
+
+    public SleepAnswer(int minSleepTime, int maxSleepTime) {
+      this.minSleepTime = minSleepTime;
       this.maxSleepTime = maxSleepTime;
     }
     
@@ -652,7 +658,7 @@ public abstract class GenericTestUtils {
     public Object answer(InvocationOnMock invocation) throws Throwable {
       boolean interrupted = false;
       try {
-        Thread.sleep(r.nextInt(maxSleepTime));
+        Thread.sleep(r.nextInt(maxSleepTime) + minSleepTime);
       } catch (InterruptedException ie) {
         interrupted = true;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42307e3c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 4986027..1cdb159 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -164,6 +164,8 @@ public class BlockManager implements BlockStatsMXBean {
   private static final String QUEUE_REASON_FUTURE_GENSTAMP =
     "generation stamp is in the future";
 
+  private static final long BLOCK_RECOVERY_TIMEOUT_MULTIPLIER = 30;
+
   private final Namesystem namesystem;
 
   private final BlockManagerSafeMode bmSafeMode;
@@ -353,6 +355,9 @@ public class BlockManager implements BlockStatsMXBean {
   @VisibleForTesting
   final PendingReconstructionBlocks pendingReconstruction;
 
+  /** Stores information about block recovery attempts. */
+  private final PendingRecoveryBlocks pendingRecoveryBlocks;
+
   /** The maximum number of replicas allowed for a block */
   public final short maxReplication;
   /**
@@ -549,6 +554,12 @@ public class BlockManager implements BlockStatsMXBean {
     }
     this.minReplicationToBeInMaintenance = (short)minMaintenanceR;
 
+    long heartbeatIntervalSecs = conf.getTimeDuration(
+        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
+        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS);
+    long blockRecoveryTimeout = getBlockRecoveryTimeout(heartbeatIntervalSecs);
+    pendingRecoveryBlocks = new PendingRecoveryBlocks(blockRecoveryTimeout);
+
     this.blockReportLeaseManager = new BlockReportLeaseManager(conf);
 
     bmSafeMode = new BlockManagerSafeMode(this, namesystem, haEnabled, conf);
@@ -4736,6 +4747,25 @@ public class BlockManager implements BlockStatsMXBean {
     }
   }
 
+  /**
+   * Notification of a successful block recovery.
+   * @param block for which the recovery succeeded
+   */
+  public void successfulBlockRecovery(BlockInfo block) {
+    pendingRecoveryBlocks.remove(block);
+  }
+
+  /**
+   * Checks whether a recovery attempt has been made for the given block.
+   * If so, checks whether that attempt has timed out.
+   * @param b block for which recovery is being attempted
+   * @return true if no recovery attempt has been made or
+   *         the previous attempt timed out
+   */
+  public boolean addBlockRecoveryAttempt(BlockInfo b) {
+    return pendingRecoveryBlocks.add(b);
+  }
+
   @VisibleForTesting
   public void flushBlockOps() throws IOException {
     runBlockOp(new Callable<Void>(){
@@ -4863,4 +4893,14 @@ public class BlockManager implements BlockStatsMXBean {
     }
     return i;
   }
+
+  private static long getBlockRecoveryTimeout(long heartbeatIntervalSecs) {
+    return TimeUnit.SECONDS.toMillis(heartbeatIntervalSecs *
+        BLOCK_RECOVERY_TIMEOUT_MULTIPLIER);
+  }
+
+  @VisibleForTesting
+  public void setBlockRecoveryTimeout(long blockRecoveryTimeout) {
+    pendingRecoveryBlocks.setRecoveryTimeoutInterval(blockRecoveryTimeout);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42307e3c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingRecoveryBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingRecoveryBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingRecoveryBlocks.java
new file mode 100644
index 0000000..3f5f27c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingRecoveryBlocks.java
@@ -0,0 +1,143 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdfs.util.LightWeightHashSet;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * PendingRecoveryBlocks tracks recovery attempts for each block and their
+ * timeouts to ensure we do not have multiple recoveries at the same time
+ * and retry only after the timeout for a recovery has expired.
+ */
+class PendingRecoveryBlocks {
+  private static final Logger LOG = BlockManager.LOG;
+
+  /** List of recovery attempts per block and the time they expire. */
+  private final LightWeightHashSet<BlockRecoveryAttempt> recoveryTimeouts =
+      new LightWeightHashSet<>();
+
+  /** The timeout for issuing a block recovery again.
+   * (it should be larger than the time to recover a block)
+   */
+  private long recoveryTimeoutInterval;
+
+  PendingRecoveryBlocks(long timeout) {
+    this.recoveryTimeoutInterval = timeout;
+  }
+
+  /**
+   * Remove recovery attempt for the given block.
+   * @param block whose recovery attempt to remove.
+   */
+  synchronized void remove(BlockInfo block) {
+    recoveryTimeouts.remove(new BlockRecoveryAttempt(block));
+  }
+
+  /**
+   * Checks whether a recovery attempt has been made for the given block.
+   * If so, checks whether that attempt has timed out.
+   * @param block block for which recovery is being attempted
+   * @return true if no recovery attempt has been made or
+   *         the previous attempt timed out
+   */
+  synchronized boolean add(BlockInfo block) {
+    boolean added = false;
+    long curTime = getTime();
+    BlockRecoveryAttempt recoveryAttempt =
+        recoveryTimeouts.getElement(new BlockRecoveryAttempt(block));
+
+    if (recoveryAttempt == null) {
+      BlockRecoveryAttempt newAttempt = new BlockRecoveryAttempt(
+          block, curTime + recoveryTimeoutInterval);
+      added = recoveryTimeouts.add(newAttempt);
+    } else if (recoveryAttempt.hasTimedOut(curTime)) {
+      // Previous attempt timed out, reset the timeout
+      recoveryAttempt.setTimeout(curTime + recoveryTimeoutInterval);
+      added = true;
+    } else {
+      long timeoutIn = TimeUnit.MILLISECONDS.toSeconds(
+          recoveryAttempt.timeoutAt - curTime);
+      LOG.info("Block recovery attempt for " + block + " rejected, as the " +
+          "previous attempt times out in " + timeoutIn + " seconds.");
+    }
+    return added;
+  }
+
+  /**
+   * Check whether the given block is under recovery.
+   * @param b block for which to check
+   * @return true if the given block is being recovered
+   */
+  synchronized boolean isUnderRecovery(BlockInfo b) {
+    BlockRecoveryAttempt recoveryAttempt =
+        recoveryTimeouts.getElement(new BlockRecoveryAttempt(b));
+    return recoveryAttempt != null;
+  }
+
+  long getTime() {
+    return Time.monotonicNow();
+  }
+
+  @VisibleForTesting
+  synchronized void setRecoveryTimeoutInterval(long recoveryTimeoutInterval) {
+    this.recoveryTimeoutInterval = recoveryTimeoutInterval;
+  }
+
+  /**
+   * Tracks timeout for block recovery attempt of a given block.
+   */
+  private static class BlockRecoveryAttempt {
+    private final BlockInfo blockInfo;
+    private long timeoutAt;
+
+    private BlockRecoveryAttempt(BlockInfo blockInfo) {
+      this(blockInfo, 0);
+    }
+
+    BlockRecoveryAttempt(BlockInfo blockInfo, long timeoutAt) {
+      this.blockInfo = blockInfo;
+      this.timeoutAt = timeoutAt;
+    }
+
+    boolean hasTimedOut(long currentTime) {
+      return currentTime > timeoutAt;
+    }
+
+    void setTimeout(long newTimeoutAt) {
+      this.timeoutAt = newTimeoutAt;
+    }
+
+    @Override
+    public int hashCode() {
+      return blockInfo.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj instanceof BlockRecoveryAttempt) {
+        return this.blockInfo.equals(((BlockRecoveryAttempt) obj).blockInfo);
+      }
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42307e3c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index d3d9cdc..6a890e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -3318,25 +3318,30 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             + "Removed empty last block and closed file " + src);
         return true;
       }
-      // start recovery of the last block for this file
-      long blockRecoveryId = nextGenerationStamp(
-          blockManager.isLegacyBlock(lastBlock));
-      lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
-      if(copyOnTruncate) {
-        lastBlock.setGenerationStamp(blockRecoveryId);
-      } else if(truncateRecovery) {
-        recoveryBlock.setGenerationStamp(blockRecoveryId);
-      }
-      uc.initializeBlockRecovery(lastBlock, blockRecoveryId, true);
-      leaseManager.renewLease(lease);
-      // Cannot close file right now, since the last block requires recovery.
-      // This may potentially cause infinite loop in lease recovery
-      // if there are no valid replicas on data-nodes.
-      NameNode.stateChangeLog.warn(
-                "DIR* NameSystem.internalReleaseLease: " +
+      // Start recovery of the last block for this file
+      // Only do so if there is no ongoing recovery for this block,
+      // or the previous recovery for this block timed out.
+      if (blockManager.addBlockRecoveryAttempt(lastBlock)) {
+        long blockRecoveryId = nextGenerationStamp(
+            blockManager.isLegacyBlock(lastBlock));
+        if(copyOnTruncate) {
+          lastBlock.setGenerationStamp(blockRecoveryId);
+        } else if(truncateRecovery) {
+          recoveryBlock.setGenerationStamp(blockRecoveryId);
+        }
+        uc.initializeBlockRecovery(lastBlock, blockRecoveryId, true);
+
+        // Cannot close file right now, since the last block requires recovery.
+        // This may potentially cause infinite loop in lease recovery
+        // if there are no valid replicas on data-nodes.
+        NameNode.stateChangeLog.warn(
+            "DIR* NameSystem.internalReleaseLease: " +
                 "File " + src + " has not been closed." +
-               " Lease recovery is in progress. " +
+                " Lease recovery is in progress. " +
                 "RecoveryId = " + blockRecoveryId + " for block " + lastBlock);
+      }
+      lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
+      leaseManager.renewLease(lease);
       break;
     }
     return false;
@@ -3604,6 +3609,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         // If this commit does not want to close the file, persist blocks
         FSDirWriteFileOp.persistBlocks(dir, src, iFile, false);
       }
+      blockManager.successfulBlockRecovery(storedBlock);
     } finally {
       writeUnlock("commitBlockSynchronization");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42307e3c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index da91006..2d710be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@ -3100,6 +3100,16 @@ public class MiniDFSCluster implements AutoCloseable {
     // Wait for new namenode to get registrations from all the datanodes
     waitActive(nnIndex);
   }
+
+  /**
+   * Sets the timeout for re-issuing a block recovery.
+   */
+  public void setBlockRecoveryTimeout(long timeout) {
+    for (int nnIndex = 0; nnIndex < getNumNameNodes(); nnIndex++) {
+      getNamesystem(nnIndex).getBlockManager().setBlockRecoveryTimeout(
+          timeout);
+    }
+  }
   
   protected void setupDatanodeAddress(Configuration conf, boolean setupHostsFile,
                            boolean checkDataNodeAddrConfig) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42307e3c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingRecoveryBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingRecoveryBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingRecoveryBlocks.java
new file mode 100644
index 0000000..baad89f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingRecoveryBlocks.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * This class contains unit tests for PendingRecoveryBlocks.java functionality.
+ */
+public class TestPendingRecoveryBlocks {
+
+  private PendingRecoveryBlocks pendingRecoveryBlocks;
+  private final long recoveryTimeout = 1000L;
+
+  private final BlockInfo blk1 = getBlock(1);
+  private final BlockInfo blk2 = getBlock(2);
+  private final BlockInfo blk3 = getBlock(3);
+
+  @Before
+  public void setUp() {
+    pendingRecoveryBlocks =
+        Mockito.spy(new PendingRecoveryBlocks(recoveryTimeout));
+  }
+
+  BlockInfo getBlock(long blockId) {
+    return new BlockInfoContiguous(new Block(blockId), (short) 0);
+  }
+
+  @Test
+  public void testAddDifferentBlocks() {
+    assertTrue(pendingRecoveryBlocks.add(blk1));
+    assertTrue(pendingRecoveryBlocks.isUnderRecovery(blk1));
+    assertTrue(pendingRecoveryBlocks.add(blk2));
+    assertTrue(pendingRecoveryBlocks.isUnderRecovery(blk2));
+    assertTrue(pendingRecoveryBlocks.add(blk3));
+    assertTrue(pendingRecoveryBlocks.isUnderRecovery(blk3));
+  }
+
+  @Test
+  public void testAddAndRemoveBlocks() {
+    // Add blocks
+    assertTrue(pendingRecoveryBlocks.add(blk1));
+    assertTrue(pendingRecoveryBlocks.add(blk2));
+
+    // Remove blk1
+    pendingRecoveryBlocks.remove(blk1);
+
+    // Adding back blk1 should succeed
+    assertTrue(pendingRecoveryBlocks.add(blk1));
+  }
+
+  @Test
+  public void testAddBlockWithPreviousRecoveryTimedOut() {
+    // Add blk
+    Mockito.doReturn(0L).when(pendingRecoveryBlocks).getTime();
+    assertTrue(pendingRecoveryBlocks.add(blk1));
+
+    // Should fail, has not timed out yet
+    Mockito.doReturn(recoveryTimeout / 2).when(pendingRecoveryBlocks).getTime();
+    assertFalse(pendingRecoveryBlocks.add(blk1));
+
+    // Should succeed after timing out
+    Mockito.doReturn(recoveryTimeout * 2).when(pendingRecoveryBlocks).getTime();
+    assertTrue(pendingRecoveryBlocks.add(blk1));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42307e3c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
index 311d5a6..e9bd7a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
@@ -18,7 +18,10 @@
 
 package org.apache.hadoop.hdfs.server.datanode;
 
+import org.apache.hadoop.hdfs.AppendTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
+
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
@@ -43,6 +46,7 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Random;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
@@ -94,6 +98,7 @@ import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.GenericTestUtils.SleepAnswer;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
@@ -1035,4 +1040,106 @@ public class TestBlockRecovery {
       Assert.fail("Thread failure: " + failureReason);
     }
   }
+
+  /**
+   * Test for block recovery taking longer than the heartbeat interval.
+   */
+  @Test(timeout = 300000L)
+  public void testRecoverySlowerThanHeartbeat() throws Exception {
+    tearDown(); // Stop the Mocked DN started in startup()
+
+    SleepAnswer delayer = new SleepAnswer(3000, 6000);
+    testRecoveryWithDatanodeDelayed(delayer);
+  }
+
+  /**
+   * Test for block recovery timeout. All recovery attempts will be delayed
+   * and the first attempt will be lost to trigger recovery timeout and retry.
+   */
+  @Test(timeout = 300000L)
+  public void testRecoveryTimeout() throws Exception {
+    tearDown(); // Stop the Mocked DN started in startup()
+    final Random r = new Random();
+
+    // Make sure first commitBlockSynchronization call from the DN gets lost
+    // for the recovery timeout to expire and new recovery attempt
+    // to be started.
+    SleepAnswer delayer = new SleepAnswer(3000) {
+      private final AtomicBoolean callRealMethod = new AtomicBoolean();
+
+      @Override
+      public Object answer(InvocationOnMock invocation) throws Throwable {
+        boolean interrupted = false;
+        try {
+          Thread.sleep(r.nextInt(3000) + 6000);
+        } catch (InterruptedException ie) {
+          interrupted = true;
+        }
+        try {
+          if (callRealMethod.get()) {
+            return invocation.callRealMethod();
+          }
+          callRealMethod.set(true);
+          return null;
+        } finally {
+          if (interrupted) {
+            Thread.currentThread().interrupt();
+          }
+        }
+      }
+    };
+    testRecoveryWithDatanodeDelayed(delayer);
+  }
+
+  private void testRecoveryWithDatanodeDelayed(
+      GenericTestUtils.SleepAnswer recoveryDelayer) throws Exception {
+    Configuration configuration = new HdfsConfiguration();
+    configuration.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    MiniDFSCluster cluster = null;
+
+    try {
+      cluster = new MiniDFSCluster.Builder(configuration)
+          .numDataNodes(2).build();
+      cluster.waitActive();
+      final FSNamesystem ns = cluster.getNamesystem();
+      final NameNode nn = cluster.getNameNode();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      cluster.setBlockRecoveryTimeout(TimeUnit.SECONDS.toMillis(15));
+
+      // Create a file and never close the output stream to trigger recovery
+      FSDataOutputStream out = dfs.create(new Path("/testSlowRecovery"),
+          (short) 2);
+      out.write(AppendTestUtil.randomBytes(0, 4096));
+      out.hsync();
+
+      List<DataNode> dataNodes = cluster.getDataNodes();
+      for (DataNode datanode : dataNodes) {
+        DatanodeProtocolClientSideTranslatorPB nnSpy =
+            InternalDataNodeTestUtils.spyOnBposToNN(datanode, nn);
+
+        Mockito.doAnswer(recoveryDelayer).when(nnSpy).
+            commitBlockSynchronization(
+                Mockito.any(ExtendedBlock.class), Mockito.anyInt(),
+                Mockito.anyLong(), Mockito.anyBoolean(),
+                Mockito.anyBoolean(), Mockito.any(DatanodeID[].class),
+                Mockito.any(String[].class));
+      }
+
+      // Make sure hard lease expires to trigger replica recovery
+      cluster.setLeasePeriod(100L, 100L);
+
+      // Wait for recovery to succeed
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          return ns.getCompleteBlocksTotal() > 0;
+        }
+      }, 300, 300000);
+
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42307e3c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
index dc7f47a..f9e1fd0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.Log;
@@ -278,12 +279,13 @@ public class TestPipelinesFailover {
     // Disable permissions so that another user can recover the lease.
     conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, false);
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    
+
     FSDataOutputStream stm = null;
     final MiniDFSCluster cluster = newMiniCluster(conf, 3);
     try {
       cluster.waitActive();
       cluster.transitionToActive(0);
+      cluster.setBlockRecoveryTimeout(TimeUnit.SECONDS.toMillis(1));
       Thread.sleep(500);
 
       LOG.info("Starting with NN 0 active");


---------------------------------------------------------------------
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: HDFS-12889. Router UI is missing robots.txt file. Contributed by Bharat Viswanadham.

Posted by ae...@apache.org.
HDFS-12889. Router UI is missing robots.txt file. Contributed by Bharat Viswanadham.


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

Branch: refs/heads/HDFS-7240
Commit: 44b06d34a537f8b558007cc92a5d1a8e59b5d86b
Parents: 0311cf0
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Dec 6 11:40:33 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Dec 6 11:40:33 2017 +0900

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/44b06d34/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index 65eea31..cfdadf2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -389,6 +389,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <exclude>src/main/webapps/hdfs/robots.txt</exclude>
             <exclude>src/main/webapps/journal/robots.txt</exclude>
             <exclude>src/main/webapps/secondary/robots.txt</exclude>
+            <exclude>src/main/webapps/router/robots.txt</exclude>
             <exclude>src/contrib/**</exclude>
             <exclude>src/site/resources/images/*</exclude>
             <exclude>src/main/webapps/static/bootstrap-3.0.2/**</exclude>


---------------------------------------------------------------------
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-7586. Application Placement should be done before ACL checks in ResourceManager. Contributed by Suma Shivaprasad.

Posted by ae...@apache.org.
YARN-7586. Application Placement should be done before ACL checks in ResourceManager. Contributed by Suma Shivaprasad.


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

Branch: refs/heads/HDFS-7240
Commit: f9f317b70209d20161d66a73ddea3334d2b92f96
Parents: 9f1bdaf
Author: Sunil G <su...@apache.org>
Authored: Tue Dec 5 18:28:31 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Tue Dec 5 18:28:31 2017 +0530

----------------------------------------------------------------------
 .../server/resourcemanager/RMAppManager.java    |  60 +++++++++-
 .../placement/PlacementManager.java             |   5 +-
 .../server/resourcemanager/rmapp/RMApp.java     |   8 ++
 .../server/resourcemanager/rmapp/RMAppImpl.java |  80 +++-----------
 .../server/resourcemanager/TestAppManager.java  | 110 ++++++++++++++++---
 .../applicationsmanager/MockAsm.java            |   7 ++
 .../server/resourcemanager/rmapp/MockRMApp.java |   7 ++
 7 files changed, 194 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9f317b7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.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/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
index 5e82f40..3ad2c9a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
@@ -48,6 +48,10 @@ import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.security.AccessRequest;
 import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
+
+import org.apache.hadoop.yarn.server.resourcemanager.placement
+    .ApplicationPlacementContext;
+import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
@@ -361,7 +365,24 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
       ApplicationSubmissionContext submissionContext, long submitTime,
       String user, boolean isRecovery, long startTime) throws YarnException {
 
+    ApplicationPlacementContext placementContext = null;
+
+    // We only do queue mapping when it's a new application
     if (!isRecovery) {
+      try {
+        // Do queue mapping
+        placementContext = placeApplication(rmContext,
+            submissionContext, user);
+        replaceQueueFromPlacementContext(placementContext,
+            submissionContext);
+      } catch (YarnException e) {
+        String msg = "Failed to place application " +
+            submissionContext.getApplicationId() + " to queue and specified "
+            + "queue is invalid : " + submissionContext.getQueue();
+        LOG.error(msg, e);
+        throw e;
+      }
+
       // fail the submission if configured application timeout value is invalid
       RMServerUtils.validateApplicationTimeouts(
           submissionContext.getApplicationTimeouts());
@@ -413,7 +434,8 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
             submissionContext.getQueue(),
             submissionContext, this.scheduler, this.masterService,
             submitTime, submissionContext.getApplicationType(),
-            submissionContext.getApplicationTags(), amReqs, startTime);
+            submissionContext.getApplicationTags(), amReqs, placementContext,
+            startTime);
     // Concurrent app submissions with same applicationId will fail here
     // Concurrent app submissions with different applicationIds will not
     // influence each other
@@ -758,4 +780,40 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
           + "' with below exception:" + ex.getMessage());
     }
   }
+
+  @VisibleForTesting
+  ApplicationPlacementContext placeApplication(RMContext rmContext,
+      ApplicationSubmissionContext context, String user) throws YarnException {
+    ApplicationPlacementContext placementContext = null;
+    PlacementManager placementManager = rmContext.getQueuePlacementManager();
+
+    if (placementManager != null) {
+      placementContext = placementManager.placeApplication(context, user);
+    } else{
+      if ( context.getQueue() == null || context.getQueue().isEmpty()) {
+        final String msg = "Queue Placement Manager is not set. Cannot place "
+            + "application : " + context.getApplicationId() + " to queue and "
+            + "specified queue is invalid " + context.getQueue();
+        LOG.error(msg);
+        throw new YarnException(msg);
+      }
+    }
+
+    return placementContext;
+  }
+
+  void replaceQueueFromPlacementContext(
+      ApplicationPlacementContext placementContext,
+      ApplicationSubmissionContext context) {
+    // Set it to ApplicationSubmissionContext
+    //apply queue mapping only to new application submissions
+    if (placementContext != null && !StringUtils.equalsIgnoreCase(
+        context.getQueue(), placementContext.getQueue())) {
+      LOG.info("Placed application=" + context.getApplicationId() +
+          " to queue=" + placementContext.getQueue() + ", original queue="
+          + context
+          .getQueue());
+      context.setQueue(placementContext.getQueue());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9f317b7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementManager.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/placement/PlacementManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementManager.java
index c006738..5fa7723 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementManager.java
@@ -72,8 +72,9 @@ public class PlacementManager {
 
       // Failed to get where to place application
       if (null == placement && null == asc.getQueue()) {
-        String msg = "Failed to get where to place application=" + asc
-            .getApplicationId();
+        String msg = "Failed to place application " +
+            asc.getApplicationId() + " to queue and specified "
+            + "queue is invalid : " + asc.getQueue();
         LOG.error(msg);
         throw new YarnException(msg);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9f317b7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
index 8583789..b357d91 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
@@ -43,6 +43,8 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
 import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
+import org.apache.hadoop.yarn.server.resourcemanager.placement
+    .ApplicationPlacementContext;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 
@@ -303,4 +305,10 @@ public interface RMApp extends EventHandler<RMAppEvent> {
    * @return True/False to confirm whether app is in final states
    */
   boolean isAppInCompletedStates();
+
+  /**
+   * Get the application -> queue placement context
+   * @return ApplicationPlacementContext
+   */
+  ApplicationPlacementContext getApplicationPlacementContext();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9f317b7/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 0266b83..38f666b 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
@@ -67,7 +67,6 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
@@ -83,8 +82,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.blacklist.BlacklistManager;
 import org.apache.hadoop.yarn.server.resourcemanager.blacklist.DisabledBlacklistManager;
 import org.apache.hadoop.yarn.server.resourcemanager.blacklist.SimpleBlacklistManager;
-import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
-import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
+
+import org.apache.hadoop.yarn.server.resourcemanager.placement
+    .ApplicationPlacementContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
@@ -204,6 +204,8 @@ public class RMAppImpl implements RMApp, Recoverable {
   
   private CallerContext callerContext;
 
+  private ApplicationPlacementContext placementContext;
+
   Object transitionTodo;
 
   private Priority applicationPriority;
@@ -417,7 +419,7 @@ public class RMAppImpl implements RMApp, Recoverable {
       List<ResourceRequest> amReqs) {
     this(applicationId, rmContext, config, name, user, queue, submissionContext,
       scheduler, masterService, submitTime, applicationType, applicationTags,
-      amReqs, -1);
+      amReqs, null, -1);
   }
 
   public RMAppImpl(ApplicationId applicationId, RMContext rmContext,
@@ -425,7 +427,8 @@ public class RMAppImpl implements RMApp, Recoverable {
       ApplicationSubmissionContext submissionContext, YarnScheduler scheduler,
       ApplicationMasterService masterService, long submitTime,
       String applicationType, Set<String> applicationTags,
-      List<ResourceRequest> amReqs, long startTime) {
+      List<ResourceRequest> amReqs, ApplicationPlacementContext
+      placementContext, long startTime) {
 
     this.systemClock = SystemClock.getInstance();
 
@@ -484,6 +487,8 @@ public class RMAppImpl implements RMApp, Recoverable {
 
     this.callerContext = CallerContext.getCurrent();
 
+    this.placementContext = placementContext;
+
     long localLogAggregationStatusTimeout =
         conf.getLong(YarnConfiguration.LOG_AGGREGATION_STATUS_TIME_OUT_MS,
           YarnConfiguration.DEFAULT_LOG_AGGREGATION_STATUS_TIME_OUT_MS);
@@ -1098,22 +1103,12 @@ public class RMAppImpl implements RMApp, Recoverable {
         }
       }
 
-      ApplicationPlacementContext placementContext = null;
-      try {
-        placementContext = placeApplication(app.rmContext,
-            app.submissionContext, app.user);
-      } catch (Exception e) {
-        String msg = "Failed to place application to queue :" + e.getMessage();
-        app.diagnostics.append(msg);
-        LOG.error(msg, e);
-      }
-
       // No existent attempts means the attempt associated with this app was not
       // started or started but not yet saved.
       if (app.attempts.isEmpty()) {
         app.scheduler.handle(
             new AppAddedSchedulerEvent(app.user, app.submissionContext, false,
-                app.applicationPriority, placementContext));
+                app.applicationPriority, app.placementContext));
         return RMAppState.SUBMITTED;
       }
 
@@ -1121,7 +1116,7 @@ public class RMAppImpl implements RMApp, Recoverable {
       // knows applications before AM or NM re-registers.
       app.scheduler.handle(
           new AppAddedSchedulerEvent(app.user, app.submissionContext, true,
-              app.applicationPriority, placementContext));
+              app.applicationPriority, app.placementContext));
 
       // recover attempts
       app.recoverAppAttempts();
@@ -1137,20 +1132,9 @@ public class RMAppImpl implements RMApp, Recoverable {
       RMAppTransition {
     @Override
     public void transition(RMAppImpl app, RMAppEvent event) {
-      ApplicationPlacementContext placementContext = null;
-      try {
-        placementContext = placeApplication(app.rmContext,
-            app.submissionContext, app.user);
-        replaceQueueFromPlacementContext(placementContext,
-            app.submissionContext);
-      } catch (YarnException e) {
-        String msg = "Failed to place application to queue :" + e.getMessage();
-        app.diagnostics.append(msg);
-        LOG.error(msg, e);
-      }
       app.handler.handle(
           new AppAddedSchedulerEvent(app.user, app.submissionContext, false,
-              app.applicationPriority, placementContext));
+              app.applicationPriority, app.placementContext));
       // send the ATS create Event
       app.sendATSCreateEvent();
     }
@@ -1624,6 +1608,11 @@ public class RMAppImpl implements RMApp, Recoverable {
         || appState == RMAppState.KILLING;
   }
 
+  @Override
+  public ApplicationPlacementContext getApplicationPlacementContext() {
+    return placementContext;
+  }
+
   public RMAppState getRecoveredFinalState() {
     return this.recoveredFinalState;
   }
@@ -2046,37 +2035,4 @@ public class RMAppImpl implements RMApp, Recoverable {
     this.submissionContext.setAMContainerSpec(null);
     this.submissionContext.setLogAggregationContext(null);
   }
-
-  @VisibleForTesting
-  static ApplicationPlacementContext placeApplication(RMContext rmContext,
-      ApplicationSubmissionContext context, String user) throws YarnException {
-
-    ApplicationPlacementContext placementContext = null;
-    PlacementManager placementManager = rmContext.getQueuePlacementManager();
-
-    if (placementManager != null) {
-      placementContext = placementManager.placeApplication(context, user);
-    } else{
-      LOG.error(
-          "Queue Placement Manager is null. Cannot place application :" + " "
-              + context.getApplicationId() + " to queue ");
-    }
-
-    return placementContext;
-  }
-
-  static void replaceQueueFromPlacementContext(
-      ApplicationPlacementContext placementContext,
-      ApplicationSubmissionContext context) {
-    // Set it to ApplicationSubmissionContext
-    //apply queue mapping only to new application submissions
-    if (placementContext != null && !StringUtils.equals(context.getQueue(),
-        placementContext.getQueue())) {
-      LOG.info("Placed application=" + context.getApplicationId() + " to queue="
-          + placementContext.getQueue() + ", original queue=" + context
-          .getQueue());
-      context.setQueue(placementContext.getQueue());
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9f317b7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.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/TestAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
index 9445fa6..f56de0f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.isA;
@@ -84,6 +85,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptI
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
@@ -108,7 +111,10 @@ import com.google.common.collect.Maps;
 
 public class TestAppManager{
   private Log LOG = LogFactory.getLog(TestAppManager.class);
-  private static RMAppEventType appEventType = RMAppEventType.KILL; 
+  private static RMAppEventType appEventType = RMAppEventType.KILL;
+
+  private static String USER = "user_";
+  private static String USER0 = USER + 0;
 
   public synchronized RMAppEventType getAppEventType() {
     return appEventType;
@@ -267,6 +273,70 @@ public class TestAppManager{
     setupDispatcher(rmContext, conf);
   }
 
+  @Test
+  public void testQueueSubmitWithACLsEnabledWithQueueMapping()
+      throws IOException, YarnException, InterruptedException {
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    conf.set(PREFIX + "root.queues", "default,test");
+
+    conf.setFloat(PREFIX + "root.default.capacity", 50.0f);
+    conf.setFloat(PREFIX + "root.default.maximum-capacity", 100.0f);
+
+    conf.setFloat(PREFIX + "root.test.capacity", 50.0f);
+    conf.setFloat(PREFIX + "root.test.maximum-capacity", 100.0f);
+
+    conf.set(PREFIX + "root.acl_submit_applications", " ");
+    conf.set(PREFIX + "root.acl_administer_queue", " ");
+
+    conf.set(PREFIX + "root.default.acl_submit_applications", " ");
+    conf.set(PREFIX + "root.default.acl_administer_queue", " ");
+
+    conf.set(PREFIX + "root.test.acl_submit_applications", "test");
+    conf.set(PREFIX + "root.test.acl_administer_queue", "test");
+
+    conf.set(YarnConfiguration.YARN_ACL_ENABLE, "true");
+
+    // Setup a PlacementManager returns a new queue
+    PlacementManager placementMgr = mock(PlacementManager.class);
+    doAnswer(new Answer<ApplicationPlacementContext>() {
+
+      @Override
+      public ApplicationPlacementContext answer(InvocationOnMock invocation)
+          throws Throwable {
+        return new ApplicationPlacementContext("test");
+      }
+
+    }).when(placementMgr).placeApplication(
+        any(ApplicationSubmissionContext.class), any(String.class));
+
+    asContext.setQueue("oldQueue");
+
+    MockRM newMockRM = new MockRM(conf);
+    RMContext newMockRMContext = newMockRM.getRMContext();
+    newMockRMContext.setQueuePlacementManager(placementMgr);
+    ApplicationMasterService masterService =
+        new ApplicationMasterService(newMockRMContext,
+            newMockRMContext.getScheduler());
+
+    TestRMAppManager newAppMonitor = new TestRMAppManager(newMockRMContext,
+        new ClientToAMTokenSecretManagerInRM(),
+        newMockRMContext.getScheduler(), masterService,
+        new ApplicationACLsManager(conf), conf);
+
+    //only user test has permission to submit to 'test' queue
+    newAppMonitor.submitApplication(asContext, "test");
+
+    try {
+      //should fail since user does not have permission to submit to queue
+      // 'test'
+      newAppMonitor.submitApplication(asContext, "test1");
+    } catch(YarnException e) {
+      assertTrue(e.getCause() instanceof AccessControlException);
+    }
+  }
+
   @After
   public void tearDown() {
     setAppEventType(RMAppEventType.KILL);
@@ -328,7 +398,7 @@ public class TestAppManager{
     } catch (Exception e) {
       e.printStackTrace();
       if (e instanceof YarnException) {
-        Assert.assertTrue(e.getCause() instanceof AccessControlException);
+        assertTrue(e.getCause() instanceof AccessControlException);
       } else {
         Assert.fail("Yarn exception is expected : " + e.getMessage());
       }
@@ -718,7 +788,7 @@ public class TestAppManager{
           " Tokens are invalid.");
     } catch (YarnException e) {
       // Exception is expected
-      Assert.assertTrue("The thrown exception is not" +
+      assertTrue("The thrown exception is not" +
           " java.io.EOFException",
           e.getMessage().contains("java.io.EOFException"));
     }
@@ -778,22 +848,25 @@ public class TestAppManager{
     ApplicationId appId = MockApps.newAppID(0);
     asContext.setApplicationId(appId);
     RMApp appOrig = rmContext.getRMApps().get(appId);
-    Assert.assertTrue("app name matches but shouldn't", "testApp1" != appOrig.getName());
+    assertTrue("app name matches "
+        + "but shouldn't", "testApp1" != appOrig.getName());
 
     // our testApp1 should be rejected and original app with same id should be left in place
     try {
       appMonitor.submitApplication(asContext, "test");
       Assert.fail("Exception is expected when applicationId is duplicate.");
     } catch (YarnException e) {
-      Assert.assertTrue("The thrown exception is not the expectd one.",
+      assertTrue("The thrown exception is not the expectd one.",
           e.getMessage().contains("Cannot add a duplicate!"));
     }
 
     // make sure original app didn't get removed
     RMApp app = rmContext.getRMApps().get(appId);
     Assert.assertNotNull("app is null", app);
-    Assert.assertEquals("app id doesn't match", appId, app.getApplicationId());
-    Assert.assertEquals("app state doesn't match", RMAppState.FINISHED, app.getState());
+    Assert.assertEquals("app id doesn't match",
+        appId, app.getApplicationId());
+    Assert.assertEquals("app state doesn't match",
+        RMAppState.FINISHED, app.getState());
   }
 
   @SuppressWarnings("deprecation")
@@ -811,7 +884,7 @@ public class TestAppManager{
       // Exception is expected
       // TODO Change this to assert the expected exception type - post YARN-142
       // sub-task related to specialized exceptions.
-      Assert.assertTrue("The thrown exception is not" +
+      assertTrue("The thrown exception is not" +
           " InvalidResourceRequestException",
           e.getMessage().contains("Invalid resource request"));
     }
@@ -844,16 +917,16 @@ public class TestAppManager{
     Assert.assertFalse(msg.contains("\r"));
 
     String escaped = "\\n\\n\\r\\r";
-    Assert.assertTrue(msg.contains("Multiline" + escaped +"AppName"));
-    Assert.assertTrue(msg.contains("Multiline" + escaped +"UserName"));
-    Assert.assertTrue(msg.contains("Multiline" + escaped +"QueueName"));
-    Assert.assertTrue(msg.contains("submitTime=1000"));
-    Assert.assertTrue(msg.contains("memorySeconds=16384"));
-    Assert.assertTrue(msg.contains("vcoreSeconds=64"));
-    Assert.assertTrue(msg.contains("preemptedAMContainers=1"));
-    Assert.assertTrue(msg.contains("preemptedNonAMContainers=10"));
-    Assert.assertTrue(msg.contains("preemptedResources=<memory:1234\\, vCores:56>"));
-    Assert.assertTrue(msg.contains("applicationType=MAPREDUCE"));
+    assertTrue(msg.contains("Multiline" + escaped +"AppName"));
+    assertTrue(msg.contains("Multiline" + escaped +"UserName"));
+    assertTrue(msg.contains("Multiline" + escaped +"QueueName"));
+    assertTrue(msg.contains("submitTime=1000"));
+    assertTrue(msg.contains("memorySeconds=16384"));
+    assertTrue(msg.contains("vcoreSeconds=64"));
+    assertTrue(msg.contains("preemptedAMContainers=1"));
+    assertTrue(msg.contains("preemptedNonAMContainers=10"));
+    assertTrue(msg.contains("preemptedResources=<memory:1234\\, vCores:56>"));
+    assertTrue(msg.contains("applicationType=MAPREDUCE"));
  }
 
   @Test
@@ -952,4 +1025,5 @@ public class TestAppManager{
     }
     return cloneReqs;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9f317b7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
index 9ef48db..2aca375 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
@@ -46,6 +46,8 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
 import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
+import org.apache.hadoop.yarn.server.resourcemanager.placement
+    .ApplicationPlacementContext;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
@@ -244,6 +246,11 @@ public abstract class MockAsm extends MockApps {
     }
 
     @Override
+    public ApplicationPlacementContext getApplicationPlacementContext() {
+      throw new UnsupportedOperationException("Not supported yet.");
+    }
+
+    @Override
     public CollectorInfo getCollectorInfo() {
       throw new UnsupportedOperationException("Not supported yet.");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9f317b7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
index 6c64a67..7567599 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
@@ -45,6 +45,8 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPB
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
 import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
+import org.apache.hadoop.yarn.server.resourcemanager.placement
+    .ApplicationPlacementContext;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 
@@ -328,6 +330,11 @@ public class MockRMApp implements RMApp {
   }
 
   @Override
+  public ApplicationPlacementContext getApplicationPlacementContext() {
+    return null;
+  }
+
+  @Override
   public CollectorInfo getCollectorInfo() {
     throw new UnsupportedOperationException("Not supported yet.");
   }


---------------------------------------------------------------------
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: HADOOP-14475 Metrics of S3A don't print out when enabled. Contributed by Younger and Sean Mackrory.

Posted by ae...@apache.org.
HADOOP-14475 Metrics of S3A don't print out when enabled. Contributed by Younger 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/6555af81
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6555af81
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6555af81

Branch: refs/heads/HDFS-7240
Commit: 6555af81a26b0b72ec3bee7034e01f5bd84b1564
Parents: a957f1c
Author: Aaron Fabbri <fa...@apache.org>
Authored: Tue Dec 5 11:06:32 2017 -0800
Committer: Aaron Fabbri <fa...@apache.org>
Committed: Tue Dec 5 11:06:32 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |   3 +
 .../hadoop/fs/s3a/S3AInstrumentation.java       | 107 ++++++++++++++++---
 .../src/site/markdown/tools/hadoop-aws/index.md |  38 ++++++-
 .../apache/hadoop/fs/s3a/ITestS3AMetrics.java   |  51 +++++++++
 4 files changed, 186 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6555af81/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 fba8ea1..63a4349 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
@@ -78,6 +78,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.ListeningExecutorService;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -2423,6 +2424,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
         metadataStore.close();
         metadataStore = null;
       }
+      IOUtils.closeQuietly(instrumentation);
+      instrumentation = null;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6555af81/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
index ef5a434..0fbcc00 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs.s3a;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -29,8 +30,10 @@ import org.apache.hadoop.metrics2.MetricStringBuilder;
 import org.apache.hadoop.metrics2.MetricsCollector;
 import org.apache.hadoop.metrics2.MetricsInfo;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsSystem;
 import org.apache.hadoop.metrics2.MetricsTag;
-import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
 import org.apache.hadoop.metrics2.lib.Interns;
 import org.apache.hadoop.metrics2.lib.MetricsRegistry;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
@@ -58,16 +61,49 @@ import static org.apache.hadoop.fs.s3a.Statistic.*;
  * the operations to increment/query metric values are designed to handle
  * lookup failures.
  */
-@Metrics(about = "Metrics for S3a", context = "S3AFileSystem")
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class S3AInstrumentation {
+public class S3AInstrumentation implements Closeable, MetricsSource {
   private static final Logger LOG = LoggerFactory.getLogger(
       S3AInstrumentation.class);
 
-  public static final String CONTEXT = "S3AFileSystem";
+  private static final String METRICS_SOURCE_BASENAME = "S3AMetrics";
+
+  /**
+   * {@value #METRICS_SYSTEM_NAME} The name of the s3a-specific metrics
+   * system instance used for s3a metrics.
+   */
+  public static final String METRICS_SYSTEM_NAME = "s3a-file-system";
+
+  /**
+   * {@value #CONTEXT} Currently all s3a metrics are placed in a single
+   * "context". Distinct contexts may be used in the future.
+   */
+  public static final String CONTEXT = "s3aFileSystem";
+
+  /**
+   * {@value #METRIC_TAG_FILESYSTEM_ID} The name of a field added to metrics
+   * records that uniquely identifies a specific FileSystem instance.
+   */
+  public static final String METRIC_TAG_FILESYSTEM_ID = "s3aFileSystemId";
+
+  /**
+   * {@value #METRIC_TAG_BUCKET} The name of a field added to metrics records
+   * that indicates the hostname portion of the FS URL.
+   */
+  public static final String METRIC_TAG_BUCKET = "bucket";
+
+  // metricsSystemLock must be used to synchronize modifications to
+  // metricsSystem and the following counters.
+  private static Object metricsSystemLock = new Object();
+  private static MetricsSystem metricsSystem = null;
+  private static int metricsSourceNameCounter = 0;
+  private static int metricsSourceActiveCounter = 0;
+
+  private String metricsSourceName;
+
   private final MetricsRegistry registry =
-      new MetricsRegistry("S3AFileSystem").setContext(CONTEXT);
+      new MetricsRegistry("s3aFileSystem").setContext(CONTEXT);
   private final MutableCounterLong streamOpenOperations;
   private final MutableCounterLong streamCloseOperations;
   private final MutableCounterLong streamClosed;
@@ -146,7 +182,6 @@ public class S3AInstrumentation {
       STORE_IO_THROTTLED
   };
 
-
   private static final Statistic[] GAUGES_TO_CREATE = {
       OBJECT_PUT_REQUESTS_ACTIVE,
       OBJECT_PUT_BYTES_PENDING,
@@ -157,12 +192,10 @@ public class S3AInstrumentation {
 
   public S3AInstrumentation(URI name) {
     UUID fileSystemInstanceId = UUID.randomUUID();
-    registry.tag("FileSystemId",
-        "A unique identifier for the FS ",
-        fileSystemInstanceId.toString() + "-" + name.getHost());
-    registry.tag("fsURI",
-        "URI of this filesystem",
-        name.toString());
+    registry.tag(METRIC_TAG_FILESYSTEM_ID,
+        "A unique identifier for the instance",
+        fileSystemInstanceId.toString());
+    registry.tag(METRIC_TAG_BUCKET, "Hostname from the FS URL", name.getHost());
     streamOpenOperations = streamCounter(STREAM_OPENED);
     streamCloseOperations = streamCounter(STREAM_CLOSE_OPERATIONS);
     streamClosed = streamCounter(STREAM_CLOSED);
@@ -204,6 +237,39 @@ public class S3AInstrumentation {
         "ops", "latency", interval);
     quantiles(S3GUARD_METADATASTORE_THROTTLE_RATE,
         "events", "frequency (Hz)", interval);
+
+    registerAsMetricsSource(name);
+  }
+
+  @VisibleForTesting
+  public MetricsSystem getMetricsSystem() {
+    synchronized (metricsSystemLock) {
+      if (metricsSystem == null) {
+        metricsSystem = new MetricsSystemImpl();
+        metricsSystem.init(METRICS_SYSTEM_NAME);
+      }
+    }
+    return metricsSystem;
+  }
+
+  /**
+   * Register this instance as a metrics source.
+   * @param name s3a:// URI for the associated FileSystem instance
+   */
+  private void registerAsMetricsSource(URI name) {
+    int number;
+    synchronized(metricsSystemLock) {
+      getMetricsSystem();
+
+      metricsSourceActiveCounter++;
+      number = ++metricsSourceNameCounter;
+    }
+    String msName = METRICS_SOURCE_BASENAME + number;
+    if (number > 1) {
+      msName = msName + number;
+    }
+    metricsSourceName = msName + "-" + name.getHost();
+    metricsSystem.register(metricsSourceName, "", this);
   }
 
   /**
@@ -560,6 +626,23 @@ public class S3AInstrumentation {
     streamBytesDiscardedInAbort.incr(statistics.bytesDiscardedInAbort);
   }
 
+  @Override
+  public void getMetrics(MetricsCollector collector, boolean all) {
+    registry.snapshot(collector.addRecord(registry.info().name()), true);
+  }
+
+  public void close() {
+    synchronized (metricsSystemLock) {
+      metricsSystem.unregisterSource(metricsSourceName);
+      int activeSources = --metricsSourceActiveCounter;
+      if (activeSources == 0) {
+        metricsSystem.publishMetricsNow();
+        metricsSystem.shutdown();
+        metricsSystem = null;
+      }
+    }
+  }
+
   /**
    * Statistics updated by an input stream during its actual operation.
    * These counters not thread-safe and are for use in a single instance

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6555af81/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 aaf9f23..a8d2e48 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
@@ -1,4 +1,3 @@
-
 <!---
   Licensed under the Apache License, Version 2.0 (the "License");
   you may not use this file except in compliance with the License.
@@ -1599,6 +1598,43 @@ basis.
 to set fadvise policies on input streams. Once implemented,
 this will become the supported mechanism used for configuring the input IO policy.
 
+##<a name="metrics"></a>Metrics
+
+S3A metrics can be monitored through Hadoop's metrics2 framework. S3A creates
+its own metrics system called s3a-file-system, and each instance of the client
+will create its own metrics source, named with a JVM-unique numerical ID.
+
+As a simple example, the following can be added to `hadoop-metrics2.properties`
+to write all S3A metrics to a log file every 10 seconds:
+
+    s3a-file-system.sink.my-metrics-config.class=org.apache.hadoop.metrics2.sink.FileSink
+    s3a-file-system.sink.my-metrics-config.filename=/var/log/hadoop-yarn/s3a-metrics.out
+    *.period=10
+
+Lines in that file will be structured like the following:
+
+    1511208770680 s3aFileSystem.s3aFileSystem: Context=s3aFileSystem, s3aFileSystemId=892b02bb-7b30-4ffe-80ca-3a9935e1d96e, bucket=bucket,
+    Hostname=hostname-1.hadoop.apache.com, files_created=1, files_copied=2, files_copied_bytes=10000, files_deleted=5, fake_directories_deleted=3,
+    directories_created=3, directories_deleted=0, ignored_errors=0, op_copy_from_local_file=0, op_exists=0, op_get_file_status=15, op_glob_status=0,
+    op_is_directory=0, op_is_file=0, op_list_files=0, op_list_located_status=0, op_list_status=3, op_mkdirs=1, op_rename=2, object_copy_requests=0,
+    object_delete_requests=6, object_list_requests=23, object_continue_list_requests=0, object_metadata_requests=46, object_multipart_aborted=0,
+    object_put_bytes=0, object_put_requests=4, object_put_requests_completed=4, stream_write_failures=0, stream_write_block_uploads=0,
+    stream_write_block_uploads_committed=0, stream_write_block_uploads_aborted=0, stream_write_total_time=0, stream_write_total_data=0,
+    s3guard_metadatastore_put_path_request=10, s3guard_metadatastore_initialization=0, object_put_requests_active=0, object_put_bytes_pending=0,
+    stream_write_block_uploads_active=0, stream_write_block_uploads_pending=0, stream_write_block_uploads_data_pending=0,
+    S3guard_metadatastore_put_path_latencyNumOps=0, S3guard_metadatastore_put_path_latency50thPercentileLatency=0,
+    S3guard_metadatastore_put_path_latency75thPercentileLatency=0, S3guard_metadatastore_put_path_latency90thPercentileLatency=0,
+    S3guard_metadatastore_put_path_latency95thPercentileLatency=0, S3guard_metadatastore_put_path_latency99thPercentileLatency=0
+
+Depending on other configuration, metrics from other systems, contexts, etc. may
+also get recorded, for example the following:
+
+    1511208770680 metricssystem.MetricsSystem: Context=metricssystem, Hostname=s3a-metrics-4.gce.cloudera.com, NumActiveSources=1, NumAllSources=1,
+    NumActiveSinks=1, NumAllSinks=0, Sink_fileNumOps=2, Sink_fileAvgTime=1.0, Sink_fileDropped=0, Sink_fileQsize=0, SnapshotNumOps=5,
+    SnapshotAvgTime=0.0, PublishNumOps=2, PublishAvgTime=0.0, DroppedPubAll=0
+
+Note that low-level metrics from the AWS SDK itself are not currently included
+in these metrics.
 
 ##<a name="further_reading"></a> Other Topics
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6555af81/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java
new file mode 100644
index 0000000..182990c
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetrics.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/**
+ * Test s3a performance metrics register and output.
+ */
+public class ITestS3AMetrics extends AbstractS3ATestBase {
+
+  @Test
+  public void testMetricsRegister()
+      throws IOException, InterruptedException {
+    S3AFileSystem fs = getFileSystem();
+    Path dest = new Path("newfile1");
+    ContractTestUtils.touch(fs, dest);
+
+    String targetMetricSource = "S3AMetrics1" + "-" + fs.getBucket();
+    assertNotNull("No metrics under test fs for " + targetMetricSource,
+        fs.getInstrumentation().getMetricsSystem()
+            .getSource(targetMetricSource));
+
+    MutableCounterLong fileCreated =
+        (MutableCounterLong) fs.getInstrumentation().getRegistry()
+            .get(Statistic.FILES_CREATED.getSymbol());
+    assertEquals("Metrics system should report single file created event",
+        1, fileCreated.value());
+  }
+}


---------------------------------------------------------------------
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: YARN-7487. Ensure volume to include GPU base libraries after created by plugin. Contributed by Wangda Tan.

Posted by ae...@apache.org.
YARN-7487. Ensure volume to include GPU base libraries after created by plugin. Contributed by Wangda Tan.


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

Branch: refs/heads/HDFS-7240
Commit: 556aea3f367bdbd4e4db601bea0ca9bf2adde063
Parents: 4653aa3
Author: Sunil G <su...@apache.org>
Authored: Fri Dec 1 13:36:28 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Fri Dec 1 13:36:28 2017 +0530

----------------------------------------------------------------------
 .../runtime/DockerLinuxContainerRuntime.java    |  63 ++++++-
 .../runtime/docker/DockerVolumeCommand.java     |  29 +++-
 .../gpu/NvidiaDockerV1CommandPlugin.java        |   2 +-
 .../container-executor/impl/utils/docker-util.c | 106 +++++++-----
 .../test/utils/test_docker_util.cc              |   5 +-
 .../runtime/TestDockerContainerRuntime.java     | 170 ++++++++++++++++---
 6 files changed, 304 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/556aea3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
index e61dc23..20359ea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
@@ -337,7 +337,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
     return false;
   }
 
-  private void runDockerVolumeCommand(DockerVolumeCommand dockerVolumeCommand,
+  private String runDockerVolumeCommand(DockerVolumeCommand dockerVolumeCommand,
       Container container) throws ContainerExecutionException {
     try {
       String commandFile = dockerClient.writeCommandToTempFile(
@@ -351,6 +351,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       LOG.info("ContainerId=" + container.getContainerId()
           + ", docker volume output for " + dockerVolumeCommand + ": "
           + output);
+      return output;
     } catch (ContainerExecutionException e) {
       LOG.error("Error when writing command to temp file, command="
               + dockerVolumeCommand,
@@ -378,15 +379,73 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
             plugin.getDockerCommandPluginInstance();
         if (dockerCommandPlugin != null) {
           DockerVolumeCommand dockerVolumeCommand =
-              dockerCommandPlugin.getCreateDockerVolumeCommand(ctx.getContainer());
+              dockerCommandPlugin.getCreateDockerVolumeCommand(
+                  ctx.getContainer());
           if (dockerVolumeCommand != null) {
             runDockerVolumeCommand(dockerVolumeCommand, container);
+
+            // After volume created, run inspect to make sure volume properly
+            // created.
+            if (dockerVolumeCommand.getSubCommand().equals(
+                DockerVolumeCommand.VOLUME_CREATE_SUB_COMMAND)) {
+              checkDockerVolumeCreated(dockerVolumeCommand, container);
+            }
           }
         }
       }
     }
   }
 
+  private void checkDockerVolumeCreated(
+      DockerVolumeCommand dockerVolumeCreationCommand, Container container)
+      throws ContainerExecutionException {
+    DockerVolumeCommand dockerVolumeInspectCommand = new DockerVolumeCommand(
+        DockerVolumeCommand.VOLUME_LS_SUB_COMMAND);
+    dockerVolumeInspectCommand.setFormat("{{.Name}},{{.Driver}}");
+    String output = runDockerVolumeCommand(dockerVolumeInspectCommand,
+        container);
+
+    // Parse output line by line and check if it matches
+    String volumeName = dockerVolumeCreationCommand.getVolumeName();
+    String driverName = dockerVolumeCreationCommand.getDriverName();
+    if (driverName == null) {
+      driverName = "local";
+    }
+
+    for (String line : output.split("\n")) {
+      line = line.trim();
+      String[] arr = line.split(",");
+      String v = arr[0].trim();
+      String d = null;
+      if (arr.length > 1) {
+        d = arr[1].trim();
+      }
+      if (d != null && volumeName.equals(v) && driverName.equals(d)) {
+        // Good we found it.
+        LOG.info(
+            "Docker volume-name=" + volumeName + " driver-name=" + driverName
+                + " already exists for container=" + container
+                .getContainerId() + ", continue...");
+        return;
+      }
+    }
+
+    // Couldn't find the volume
+    String message =
+        " Couldn't find volume=" + volumeName + " driver=" + driverName
+            + " for container=" + container.getContainerId()
+            + ", please check error message in log to understand "
+            + "why this happens.";
+    LOG.error(message);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("All docker volumes in the system, command="
+          + dockerVolumeInspectCommand.toString());
+    }
+
+    throw new ContainerExecutionException(message);
+  }
+
   private void validateContainerNetworkType(String network)
       throws ContainerExecutionException {
     if (allowedNetworks.contains(network)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/556aea3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerVolumeCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerVolumeCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerVolumeCommand.java
index a477c93..aac7685 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerVolumeCommand.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerVolumeCommand.java
@@ -27,23 +27,50 @@ import java.util.regex.Pattern;
  */
 public class DockerVolumeCommand extends DockerCommand {
   public static final String VOLUME_COMMAND = "volume";
-  public static final String VOLUME_CREATE_COMMAND = "create";
+  public static final String VOLUME_CREATE_SUB_COMMAND = "create";
+  public static final String VOLUME_LS_SUB_COMMAND = "ls";
+
   // Regex pattern for volume name
   public static final Pattern VOLUME_NAME_PATTERN = Pattern.compile(
       "[a-zA-Z0-9][a-zA-Z0-9_.-]*");
 
+  private String volumeName;
+  private String driverName;
+  private String subCommand;
+
   public DockerVolumeCommand(String subCommand) {
     super(VOLUME_COMMAND);
+    this.subCommand = subCommand;
     super.addCommandArguments("sub-command", subCommand);
   }
 
   public DockerVolumeCommand setVolumeName(String volumeName) {
     super.addCommandArguments("volume", volumeName);
+    this.volumeName = volumeName;
     return this;
   }
 
   public DockerVolumeCommand setDriverName(String driverName) {
     super.addCommandArguments("driver", driverName);
+    this.driverName = driverName;
+    return this;
+  }
+
+  public String getVolumeName() {
+    return volumeName;
+  }
+
+  public String getDriverName() {
+    return driverName;
+  }
+
+  public String getSubCommand() {
+    return subCommand;
+  }
+
+  public DockerVolumeCommand setFormat(String format) {
+    super.addCommandArguments("format", format);
     return this;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/556aea3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/gpu/NvidiaDockerV1CommandPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/gpu/NvidiaDockerV1CommandPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/gpu/NvidiaDockerV1CommandPlugin.java
index 73d7048..c2e315a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/gpu/NvidiaDockerV1CommandPlugin.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/gpu/NvidiaDockerV1CommandPlugin.java
@@ -301,7 +301,7 @@ public class NvidiaDockerV1CommandPlugin implements DockerCommandPlugin {
 
     if (newVolumeName != null) {
       DockerVolumeCommand command = new DockerVolumeCommand(
-          DockerVolumeCommand.VOLUME_CREATE_COMMAND);
+          DockerVolumeCommand.VOLUME_CREATE_SUB_COMMAND);
       command.setDriverName(volumeDriver);
       command.setVolumeName(newVolumeName);
       return command;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/556aea3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c
index e88eeac..a0138d1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c
@@ -299,29 +299,19 @@ static int value_permitted(const struct configuration* executor_cfg,
 int get_docker_volume_command(const char *command_file, const struct configuration *conf, char *out,
                                const size_t outlen) {
   int ret = 0;
-  char *driver = NULL, *volume_name = NULL, *sub_command = NULL;
+  char *driver = NULL, *volume_name = NULL, *sub_command = NULL, *format = NULL;
   struct configuration command_config = {0, NULL};
   ret = read_and_verify_command_file(command_file, DOCKER_VOLUME_COMMAND, &command_config);
   if (ret != 0) {
     return ret;
   }
   sub_command = get_configuration_value("sub-command", DOCKER_COMMAND_FILE_SECTION, &command_config);
-  if (sub_command == NULL || 0 != strcmp(sub_command, "create")) {
-    fprintf(ERRORFILE, "\"create\" is the only acceptable sub-command of volume.\n");
-    ret = INVALID_DOCKER_VOLUME_COMMAND;
-    goto cleanup;
-  }
-
-  volume_name = get_configuration_value("volume", DOCKER_COMMAND_FILE_SECTION, &command_config);
-  if (volume_name == NULL || validate_volume_name(volume_name) != 0) {
-    fprintf(ERRORFILE, "%s is not a valid volume name.\n", volume_name);
-    ret = INVALID_DOCKER_VOLUME_NAME;
-    goto cleanup;
-  }
 
-  driver = get_configuration_value("driver", DOCKER_COMMAND_FILE_SECTION, &command_config);
-  if (driver == NULL) {
-    ret = INVALID_DOCKER_VOLUME_DRIVER;
+  if ((sub_command == NULL) || ((0 != strcmp(sub_command, "create")) &&
+      (0 != strcmp(sub_command, "ls")))) {
+    fprintf(ERRORFILE, "\"create/ls\" are the only acceptable sub-command of volume, input sub_command=\"%s\"\n",
+       sub_command);
+    ret = INVALID_DOCKER_VOLUME_COMMAND;
     goto cleanup;
   }
 
@@ -338,42 +328,76 @@ int get_docker_volume_command(const char *command_file, const struct configurati
     goto cleanup;
   }
 
-  ret = add_to_buffer(out, outlen, " create");
-  if (ret != 0) {
-    goto cleanup;
-  }
+  if (0 == strcmp(sub_command, "create")) {
+    volume_name = get_configuration_value("volume", DOCKER_COMMAND_FILE_SECTION, &command_config);
+    if (volume_name == NULL || validate_volume_name(volume_name) != 0) {
+      fprintf(ERRORFILE, "%s is not a valid volume name.\n", volume_name);
+      ret = INVALID_DOCKER_VOLUME_NAME;
+      goto cleanup;
+    }
 
-  ret = add_to_buffer(out, outlen, " --name=");
-  if (ret != 0) {
-    goto cleanup;
-  }
+    driver = get_configuration_value("driver", DOCKER_COMMAND_FILE_SECTION, &command_config);
+    if (driver == NULL) {
+      ret = INVALID_DOCKER_VOLUME_DRIVER;
+      goto cleanup;
+    }
 
-  ret = add_to_buffer(out, outlen, volume_name);
-  if (ret != 0) {
-    goto cleanup;
-  }
+    ret = add_to_buffer(out, outlen, " create");
+    if (ret != 0) {
+      goto cleanup;
+    }
 
-  if (!value_permitted(conf, "docker.allowed.volume-drivers", driver)) {
-    fprintf(ERRORFILE, "%s is not permitted docker.allowed.volume-drivers\n",
-      driver);
-    ret = INVALID_DOCKER_VOLUME_DRIVER;
-    goto cleanup;
-  }
+    ret = add_to_buffer(out, outlen, " --name=");
+    if (ret != 0) {
+      goto cleanup;
+    }
 
-  ret = add_to_buffer(out, outlen, " --driver=");
-  if (ret != 0) {
-    goto cleanup;
-  }
+    ret = add_to_buffer(out, outlen, volume_name);
+    if (ret != 0) {
+      goto cleanup;
+    }
 
-  ret = add_to_buffer(out, outlen, driver);
-  if (ret != 0) {
-    goto cleanup;
+    if (!value_permitted(conf, "docker.allowed.volume-drivers", driver)) {
+      fprintf(ERRORFILE, "%s is not permitted docker.allowed.volume-drivers\n",
+        driver);
+      ret = INVALID_DOCKER_VOLUME_DRIVER;
+      goto cleanup;
+    }
+
+    ret = add_to_buffer(out, outlen, " --driver=");
+    if (ret != 0) {
+      goto cleanup;
+    }
+
+    ret = add_to_buffer(out, outlen, driver);
+    if (ret != 0) {
+      goto cleanup;
+    }
+  } else if (0 == strcmp(sub_command, "ls")) {
+    format = get_configuration_value("format", DOCKER_COMMAND_FILE_SECTION, &command_config);
+
+    ret = add_to_buffer(out, outlen, " ls");
+    if (ret != 0) {
+      goto cleanup;
+    }
+
+    if (format) {
+      ret = add_to_buffer(out, outlen, " --format=");
+      if (ret != 0) {
+        goto cleanup;
+      }
+      ret = add_to_buffer(out, outlen, format);
+      if (ret != 0) {
+        goto cleanup;
+      }
+    }
   }
 
 cleanup:
   free(driver);
   free(volume_name);
   free(sub_command);
+  free(format);
 
   // clean up out buffer
   if (ret != 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/556aea3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test_docker_util.cc
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test_docker_util.cc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test_docker_util.cc
index 96b5d40..0c1c4bf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test_docker_util.cc
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test_docker_util.cc
@@ -1132,12 +1132,15 @@ namespace ContainerExecutor {
     file_cmd_vec.push_back(std::make_pair<std::string, std::string>(
         "[docker-command-execution]\n  docker-command=volume\n  sub-command=create\n  volume=volume1 \n driver=driver1",
         "volume create --name=volume1 --driver=driver1"));
+    file_cmd_vec.push_back(std::make_pair<std::string, std::string>(
+       "[docker-command-execution]\n  docker-command=volume\n  format={{.Name}},{{.Driver}}\n  sub-command=ls",
+       "volume ls --format={{.Name}},{{.Driver}}"));
 
     std::vector<std::pair<std::string, int> > bad_file_cmd_vec;
 
     // Wrong subcommand
     bad_file_cmd_vec.push_back(std::make_pair<std::string, int>(
-        "[docker-command-execution]\n  docker-command=volume\n  sub-command=ls\n  volume=volume1 \n driver=driver1",
+        "[docker-command-execution]\n  docker-command=volume\n  sub-command=inspect\n  volume=volume1 \n driver=driver1",
         static_cast<int>(INVALID_DOCKER_VOLUME_COMMAND)));
 
     // Volume not specified

http://git-wip-us.apache.org/repos/asf/hadoop/blob/556aea3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
index 6135493..4d32427 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
@@ -1301,7 +1301,7 @@ public class TestDockerContainerRuntime {
     //single invocation expected
     //due to type erasure + mocking, this verification requires a suppress
     // warning annotation on the entire method
-    verify(mockExecutor, times(1))
+    verify(mockExecutor, times(2))
         .executePrivilegedOperation(anyList(), opCaptor.capture(), any(
             File.class), anyMap(), anyBoolean(), anyBoolean());
 
@@ -1309,7 +1309,9 @@ public class TestDockerContainerRuntime {
     // hence, reset mock here
     Mockito.reset(mockExecutor);
 
-    PrivilegedOperation op = opCaptor.getValue();
+    List<PrivilegedOperation> allCaptures = opCaptor.getAllValues();
+
+    PrivilegedOperation op = allCaptures.get(0);
     Assert.assertEquals(PrivilegedOperation.OperationType
         .RUN_DOCKER_CMD, op.getOperationType());
 
@@ -1317,14 +1319,151 @@ public class TestDockerContainerRuntime {
     FileInputStream fileInputStream = new FileInputStream(commandFile);
     String fileContent = new String(IOUtils.toByteArray(fileInputStream));
     Assert.assertEquals("[docker-command-execution]\n"
-        + "  docker-command=volume\n" + "  sub-command=create\n"
-        + "  volume=volume1\n", fileContent);
+        + "  docker-command=volume\n" + "  driver=local\n"
+        + "  sub-command=create\n" + "  volume=volume1\n", fileContent);
+    fileInputStream.close();
+
+    op = allCaptures.get(1);
+    Assert.assertEquals(PrivilegedOperation.OperationType
+        .RUN_DOCKER_CMD, op.getOperationType());
+
+    commandFile = new File(StringUtils.join(",", op.getArguments()));
+    fileInputStream = new FileInputStream(commandFile);
+    fileContent = new String(IOUtils.toByteArray(fileInputStream));
+    Assert.assertEquals("[docker-command-execution]\n"
+        + "  docker-command=volume\n" + "  format={{.Name}},{{.Driver}}\n"
+        + "  sub-command=ls\n", fileContent);
+    fileInputStream.close();
+  }
+
+  private static class MockDockerCommandPlugin implements DockerCommandPlugin {
+    private final String volume;
+    private final String driver;
+
+    public MockDockerCommandPlugin(String volume, String driver) {
+      this.volume = volume;
+      this.driver = driver;
+    }
+
+    @Override
+    public void updateDockerRunCommand(DockerRunCommand dockerRunCommand,
+        Container container) throws ContainerExecutionException {
+      dockerRunCommand.setVolumeDriver("driver-1");
+      dockerRunCommand.addReadOnlyMountLocation("/source/path",
+          "/destination/path", true);
+    }
+
+    @Override
+    public DockerVolumeCommand getCreateDockerVolumeCommand(Container container)
+        throws ContainerExecutionException {
+      return new DockerVolumeCommand("create").setVolumeName(volume)
+          .setDriverName(driver);
+    }
+
+    @Override
+    public DockerVolumeCommand getCleanupDockerVolumesCommand(
+        Container container) throws ContainerExecutionException {
+      return null;
+    }
+  }
+
+  private void testDockerCommandPluginWithVolumesOutput(
+      String dockerVolumeListOutput, boolean expectFail)
+      throws PrivilegedOperationException, ContainerExecutionException,
+      IOException {
+    mockExecutor = Mockito
+        .mock(PrivilegedOperationExecutor.class);
+
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+        mockExecutor, mockCGroupsHandler);
+    when(mockExecutor
+        .executePrivilegedOperation(anyList(), any(PrivilegedOperation.class),
+            any(File.class), anyMap(), anyBoolean(), anyBoolean())).thenReturn(
+        null);
+    when(mockExecutor
+        .executePrivilegedOperation(anyList(), any(PrivilegedOperation.class),
+            any(File.class), anyMap(), anyBoolean(), anyBoolean())).thenReturn(
+        dockerVolumeListOutput);
+
+    Context nmContext = mock(Context.class);
+    ResourcePluginManager rpm = mock(ResourcePluginManager.class);
+    Map<String, ResourcePlugin> pluginsMap = new HashMap<>();
+    ResourcePlugin plugin1 = mock(ResourcePlugin.class);
+
+    // Create the docker command plugin logic, which will set volume driver
+    DockerCommandPlugin dockerCommandPlugin = new MockDockerCommandPlugin(
+        "volume1", "local");
+
+    when(plugin1.getDockerCommandPluginInstance()).thenReturn(
+        dockerCommandPlugin);
+    ResourcePlugin plugin2 = mock(ResourcePlugin.class);
+    pluginsMap.put("plugin1", plugin1);
+    pluginsMap.put("plugin2", plugin2);
+
+    when(rpm.getNameToPlugins()).thenReturn(pluginsMap);
+
+    when(nmContext.getResourcePluginManager()).thenReturn(rpm);
+
+    runtime.initialize(conf, nmContext);
+
+    ContainerRuntimeContext containerRuntimeContext = builder.build();
+
+    try {
+      runtime.prepareContainer(containerRuntimeContext);
+
+      checkVolumeCreateCommand();
+
+      runtime.launchContainer(containerRuntimeContext);
+    } catch (ContainerExecutionException e) {
+      if (expectFail) {
+        // Expected
+        return;
+      } else{
+        Assert.fail("Should successfully prepareContainers" + e);
+      }
+    }
+    if (expectFail) {
+      Assert.fail(
+          "Should fail because output is illegal");
+    }
+  }
+
+  @Test
+  public void testDockerCommandPluginCheckVolumeAfterCreation()
+      throws Exception {
+    // For following tests, we expect to have volume1,local in output
+
+    // Failure cases
+    testDockerCommandPluginWithVolumesOutput("", true);
+    testDockerCommandPluginWithVolumesOutput("volume1", true);
+    testDockerCommandPluginWithVolumesOutput("local", true);
+    testDockerCommandPluginWithVolumesOutput("volume2,local", true);
+    testDockerCommandPluginWithVolumesOutput("volum1,something", true);
+    testDockerCommandPluginWithVolumesOutput("volum1,something\nvolum2,local",
+        true);
+
+    // Success case
+    testDockerCommandPluginWithVolumesOutput("volume1,local\n", false);
+    testDockerCommandPluginWithVolumesOutput(
+        "volume_xyz,nvidia\nvolume1,local\n\n", false);
+    testDockerCommandPluginWithVolumesOutput(" volume1,  local \n", false);
+    testDockerCommandPluginWithVolumesOutput(
+        "volume_xyz,\tnvidia\n   volume1,\tlocal\n\n", false);
   }
 
+
   @Test
   public void testDockerCommandPlugin() throws Exception {
     DockerLinuxContainerRuntime runtime =
         new DockerLinuxContainerRuntime(mockExecutor, mockCGroupsHandler);
+    when(mockExecutor
+        .executePrivilegedOperation(anyList(), any(PrivilegedOperation.class),
+            any(File.class), anyMap(), anyBoolean(), anyBoolean())).thenReturn(
+        null);
+    when(mockExecutor
+        .executePrivilegedOperation(anyList(), any(PrivilegedOperation.class),
+            any(File.class), anyMap(), anyBoolean(), anyBoolean())).thenReturn(
+        "volume1,local");
 
     Context nmContext = mock(Context.class);
     ResourcePluginManager rpm = mock(ResourcePluginManager.class);
@@ -1332,27 +1471,8 @@ public class TestDockerContainerRuntime {
     ResourcePlugin plugin1 = mock(ResourcePlugin.class);
 
     // Create the docker command plugin logic, which will set volume driver
-    DockerCommandPlugin dockerCommandPlugin = new DockerCommandPlugin() {
-      @Override
-      public void updateDockerRunCommand(DockerRunCommand dockerRunCommand,
-          Container container) throws ContainerExecutionException {
-        dockerRunCommand.setVolumeDriver("driver-1");
-        dockerRunCommand.addReadOnlyMountLocation("/source/path",
-            "/destination/path", true);
-      }
-
-      @Override
-      public DockerVolumeCommand getCreateDockerVolumeCommand(Container container)
-          throws ContainerExecutionException {
-        return new DockerVolumeCommand("create").setVolumeName("volume1");
-      }
-
-      @Override
-      public DockerVolumeCommand getCleanupDockerVolumesCommand(Container container)
-          throws ContainerExecutionException {
-        return null;
-      }
-    };
+    DockerCommandPlugin dockerCommandPlugin = new MockDockerCommandPlugin(
+        "volume1", "local");
 
     when(plugin1.getDockerCommandPluginInstance()).thenReturn(
         dockerCommandPlugin);


---------------------------------------------------------------------
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: HADOOP-14976. Set HADOOP_SHELL_EXECNAME explicitly in scripts.

Posted by ae...@apache.org.
HADOOP-14976. Set HADOOP_SHELL_EXECNAME explicitly in scripts.


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

Branch: refs/heads/HDFS-7240
Commit: e00c7f78c1c00467319ce5b92e4a3ebc691d246e
Parents: 5533648
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Dec 4 21:02:01 2017 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Mon Dec 4 21:02:04 2017 -0800

----------------------------------------------------------------------
 .../hadoop-common/src/main/bin/hadoop           |  7 +++-
 .../src/test/scripts/hadoop_shell_execname.bats | 36 ++++++++++++++++++++
 .../hadoop-hdfs/src/main/bin/hdfs               |  9 +++--
 .../src/test/scripts/hadoop_shell_execname.bats | 36 ++++++++++++++++++++
 hadoop-mapreduce-project/bin/mapred             |  4 ++-
 hadoop-yarn-project/hadoop-yarn/bin/yarn        |  3 +-
 6 files changed, 90 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e00c7f78/hadoop-common-project/hadoop-common/src/main/bin/hadoop
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop b/hadoop-common-project/hadoop-common/src/main/bin/hadoop
index 3834600..1e57185 100755
--- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop
+++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop
@@ -15,8 +15,9 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+# The name of the script being executed.
+HADOOP_SHELL_EXECNAME="hadoop"
 MYNAME="${BASH_SOURCE-$0}"
-HADOOP_SHELL_EXECNAME="${MYNAME##*/}"
 
 ## @description  build up the hadoop command's usage text.
 ## @audience     public
@@ -124,6 +125,10 @@ function hadoopcmd_case
       echo "HADOOP_TOOLS_HOME='${HADOOP_TOOLS_HOME}'"
       echo "HADOOP_TOOLS_DIR='${HADOOP_TOOLS_DIR}'"
       echo "HADOOP_TOOLS_LIB_JARS_DIR='${HADOOP_TOOLS_LIB_JARS_DIR}'"
+      if [[ -n "${QATESTMODE}" ]]; then
+        echo "MYNAME=${MYNAME}"
+        echo "HADOOP_SHELL_EXECNAME=${HADOOP_SHELL_EXECNAME}"
+      fi
       exit 0
     ;;
     fs)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e00c7f78/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_shell_execname.bats
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_shell_execname.bats b/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_shell_execname.bats
new file mode 100644
index 0000000..b9c7ca8
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_shell_execname.bats
@@ -0,0 +1,36 @@
+# 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.
+
+load hadoop-functions_test_helper
+
+# Setup minimal environment to invoke the 'hadoop' command.
+hadoopcommandsetup () {
+  export HADOOP_LIBEXEC_DIR="${TMP}/libexec"
+  export HADOOP_CONF_DIR="${TMP}/conf"
+  mkdir -p "${HADOOP_LIBEXEC_DIR}"
+  echo   ". \"${BATS_TEST_DIRNAME}/../../main/bin/hadoop-functions.sh\"" > "${HADOOP_LIBEXEC_DIR}/hadoop-config.sh"
+  chmod a+rx "${HADOOP_LIBEXEC_DIR}/hadoop-config.sh"
+}
+
+# Verify that the 'hadoop' command correctly infers MYNAME and
+# HADOOP_SHELL_EXECNAME
+@test "hadoop_shell_execname" {
+  hadoopcommandsetup
+  export QATESTMODE=unittest
+  run "${BATS_TEST_DIRNAME}/../../main/bin/hadoop" envvars
+  echo ">${output}<"
+  [[ ${output} =~ MYNAME=.*/hadoop ]]
+  [[ ${output} =~ HADOOP_SHELL_EXECNAME=hadoop ]]
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e00c7f78/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 a37c39f..38be348b 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -15,8 +15,9 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+# The name of the script being executed.
+HADOOP_SHELL_EXECNAME="hdfs"
 MYNAME="${BASH_SOURCE-$0}"
-HADOOP_SHELL_EXECNAME="${MYNAME##*/}"
 
 ## @description  build up the hdfs command's usage text.
 ## @audience     public
@@ -126,6 +127,10 @@ function hdfscmd_case
       echo "HADOOP_TOOLS_HOME='${HADOOP_TOOLS_HOME}'"
       echo "HADOOP_TOOLS_DIR='${HADOOP_TOOLS_DIR}'"
       echo "HADOOP_TOOLS_LIB_JARS_DIR='${HADOOP_TOOLS_LIB_JARS_DIR}'"
+      if [[ -n "${QATESTMODE}" ]]; then
+        echo "MYNAME=${MYNAME}"
+        echo "HADOOP_SHELL_EXECNAME=${HADOOP_SHELL_EXECNAME}"
+      fi
       exit 0
     ;;
     ec)
@@ -269,4 +274,4 @@ fi
 hadoop_subcommand_opts "${HADOOP_SHELL_EXECNAME}" "${HADOOP_SUBCMD}"
 
 # everything is in globals at this point, so call the generic handler
-hadoop_generic_java_subcmd_handler
\ No newline at end of file
+hadoop_generic_java_subcmd_handler

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e00c7f78/hadoop-hdfs-project/hadoop-hdfs/src/test/scripts/hadoop_shell_execname.bats
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/scripts/hadoop_shell_execname.bats b/hadoop-hdfs-project/hadoop-hdfs/src/test/scripts/hadoop_shell_execname.bats
new file mode 100644
index 0000000..16c892e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/scripts/hadoop_shell_execname.bats
@@ -0,0 +1,36 @@
+# 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.
+
+load hdfs-functions_test_helper
+
+# Setup minimal environment to invoke the 'hdfs' command.
+hdfscommandsetup () {
+  export HADOOP_LIBEXEC_DIR="${TMP}/libexec"
+  export HADOOP_CONF_DIR="${TMP}/conf"
+  mkdir -p "${HADOOP_LIBEXEC_DIR}"
+  echo   ". \"${BATS_TEST_DIRNAME}/../../../../../hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh\"" > "${HADOOP_LIBEXEC_DIR}/hdfs-config.sh"
+  chmod a+rx "${HADOOP_LIBEXEC_DIR}/hdfs-config.sh"
+}
+
+# Verify that the 'hdfs' command correctly infers MYNAME and
+# HADOOP_SHELL_EXECNAME
+@test "hadoop_shell_execname" {
+  hdfscommandsetup
+  export QATESTMODE=unittest
+  run "${BATS_TEST_DIRNAME}/../../main/bin/hdfs" envvars
+  echo ">${output}<"
+  [[ ${output} =~ MYNAME=.*/hdfs ]]
+  [[ ${output} =~ HADOOP_SHELL_EXECNAME=hdfs ]]
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e00c7f78/hadoop-mapreduce-project/bin/mapred
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/bin/mapred b/hadoop-mapreduce-project/bin/mapred
index ce9ce21..44f6216 100755
--- a/hadoop-mapreduce-project/bin/mapred
+++ b/hadoop-mapreduce-project/bin/mapred
@@ -15,8 +15,10 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+# The name of the script being executed.
+HADOOP_SHELL_EXECNAME="mapred"
 MYNAME="${BASH_SOURCE-$0}"
-HADOOP_SHELL_EXECNAME="${MYNAME##*/}"
+
 
 ## @description  build up the mapred command's usage text.
 ## @audience     public

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e00c7f78/hadoop-yarn-project/hadoop-yarn/bin/yarn
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn b/hadoop-yarn-project/hadoop-yarn/bin/yarn
index d7b44b9..9a1dc19 100755
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn
@@ -15,8 +15,9 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+# The name of the script being executed.
+HADOOP_SHELL_EXECNAME="yarn"
 MYNAME="${BASH_SOURCE-$0}"
-HADOOP_SHELL_EXECNAME="${MYNAME##*/}"
 
 ## @description  build up the yarn command's usage text.
 ## @audience     public


---------------------------------------------------------------------
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: YARN-6124. Make SchedulingEditPolicy can be enabled / disabled / updated with RMAdmin -refreshQueues. (Zian Chen via wangda)

Posted by ae...@apache.org.
YARN-6124. Make SchedulingEditPolicy can be enabled / disabled / updated with RMAdmin -refreshQueues. (Zian Chen via wangda)

Change-Id: Id93656f3af7dcd78cafa94e33663c78d410d43c2


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

Branch: refs/heads/HDFS-7240
Commit: a63d19d36520fa55bf523483f14329756f6eadd3
Parents: 0780fdb
Author: Wangda Tan <wa...@apache.org>
Authored: Thu Nov 30 15:56:53 2017 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Nov 30 15:57:22 2017 -0800

----------------------------------------------------------------------
 .../server/resourcemanager/AdminService.java    |  21 ++-
 .../server/resourcemanager/ResourceManager.java |  31 +---
 .../monitor/SchedulingMonitor.java              |   3 +-
 .../monitor/SchedulingMonitorManager.java       | 184 +++++++++++++++++++
 .../scheduler/AbstractYarnScheduler.java        |  25 ++-
 .../scheduler/capacity/CapacityScheduler.java   |   6 +
 .../scheduler/fair/FairScheduler.java           |   6 +
 .../scheduler/fifo/FifoScheduler.java           |   6 +
 .../server/resourcemanager/RMHATestBase.java    |  30 ++-
 .../monitor/TestSchedulingMonitor.java          |  41 +++++
 ...estProportionalCapacityPreemptionPolicy.java |  22 ++-
 .../TestCapacitySchedulerLazyPreemption.java    |  36 +++-
 ...TestCapacitySchedulerSurgicalPreemption.java |  40 +++-
 13 files changed, 391 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a63d19d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.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/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
index 6c0a854..accf901 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
@@ -400,14 +400,31 @@ public class AdminService extends CompositeService implements
     }
   }
 
+  protected Configuration loadNewConfiguration()
+      throws IOException, YarnException {
+    // Retrieve yarn-site.xml in order to refresh scheduling monitor properties.
+    Configuration conf = getConfiguration(new Configuration(false),
+        YarnConfiguration.YARN_SITE_CONFIGURATION_FILE);
+    // The reason we call Configuration#size() is because when getConfiguration
+    // been called, it invokes Configuration#addResouce, which invokes
+    // Configuration#reloadConfiguration which triggers the reload process in a
+    // lazy way, the properties will only be reload when it's needed rather than
+    // reload it right after getConfiguration been called. So here we call
+    // Configuration#size() to force the Configuration#getProps been called to
+    // reload all the properties.
+    conf.size();
+    return conf;
+  }
+
   @Private
   public void refreshQueues() throws IOException, YarnException {
-    rm.getRMContext().getScheduler().reinitialize(getConfig(),
+    Configuration conf = loadNewConfiguration();
+    rm.getRMContext().getScheduler().reinitialize(conf,
         this.rm.getRMContext());
     // refresh the reservation system
     ReservationSystem rSystem = rm.getRMContext().getReservationSystem();
     if (rSystem != null) {
-      rSystem.reinitialize(getConfig(), rm.getRMContext());
+      rSystem.reinitialize(conf, rm.getRMContext());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a63d19d3/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 6f8a0a4..a0317f6 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
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.curator.framework.AuthInfo;
@@ -67,8 +68,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.metrics.NoOpSystemMetricPub
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV1Publisher;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV2Publisher;
-import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
-import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMDelegatedNodeLabelsUpdater;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
@@ -113,8 +112,6 @@ import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
 import org.eclipse.jetty.webapp.WebAppContext;
 
-import com.google.common.annotations.VisibleForTesting;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
@@ -711,8 +708,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
         }
       }
 
-      createSchedulerMonitors();
-
       masterService = createApplicationMasterService();
       addService(masterService) ;
       rmContext.setApplicationMasterService(masterService);
@@ -811,30 +806,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
       }
 
     }
-
-    protected void createSchedulerMonitors() {
-      if (conf.getBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS,
-          YarnConfiguration.DEFAULT_RM_SCHEDULER_ENABLE_MONITORS)) {
-        LOG.info("Loading policy monitors");
-        List<SchedulingEditPolicy> policies = conf.getInstances(
-            YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES,
-            SchedulingEditPolicy.class);
-        if (policies.size() > 0) {
-          for (SchedulingEditPolicy policy : policies) {
-            LOG.info("LOADING SchedulingEditPolicy:" + policy.getPolicyName());
-            // periodically check whether we need to take action to guarantee
-            // constraints
-            SchedulingMonitor mon = new SchedulingMonitor(rmContext, policy);
-            addService(mon);
-          }
-        } else {
-          LOG.warn("Policy monitors configured (" +
-              YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS +
-              ") but none specified (" +
-              YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES + ")");
-        }
-      }
-    }
   }
 
   @Private

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a63d19d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.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/monitor/SchedulingMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.java
index 2a741ed..09edb98 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.java
@@ -27,7 +27,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -58,6 +57,7 @@ public class SchedulingMonitor extends AbstractService {
   }
 
   public void serviceInit(Configuration conf) throws Exception {
+    LOG.info("Initializing SchedulingMonitor=" + getName());
     scheduleEditPolicy.init(conf, rmContext, rmContext.getScheduler());
     this.monitorInterval = scheduleEditPolicy.getMonitoringInterval();
     super.serviceInit(conf);
@@ -65,6 +65,7 @@ public class SchedulingMonitor extends AbstractService {
 
   @Override
   public void serviceStart() throws Exception {
+    LOG.info("Starting SchedulingMonitor=" + getName());
     assert !stopped : "starting when already stopped";
     ses = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() {
       public Thread newThread(Runnable r) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a63d19d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitorManager.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/monitor/SchedulingMonitorManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitorManager.java
new file mode 100644
index 0000000..0cc700d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitorManager.java
@@ -0,0 +1,184 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.monitor;
+
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Manages scheduling monitors.
+ */
+public class SchedulingMonitorManager {
+  private static final Log LOG = LogFactory.getLog(
+      SchedulingMonitorManager.class);
+
+  private Map<String, SchedulingMonitor> runningSchedulingMonitors =
+      new HashMap<>();
+  private RMContext rmContext;
+
+  private void updateSchedulingMonitors(Configuration conf,
+      boolean startImmediately) throws YarnException {
+    boolean monitorsEnabled = conf.getBoolean(
+        YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_ENABLE_MONITORS);
+
+    if (!monitorsEnabled) {
+      if (!runningSchedulingMonitors.isEmpty()) {
+        // If monitors disabled while we have some running monitors, we should
+        // stop them.
+        LOG.info("Scheduling Monitor disabled, stopping all services");
+        stopAndRemoveAll();
+      }
+
+      return;
+    }
+
+    // When monitor is enabled, loading policies
+    String[] configuredPolicies = conf.getStrings(
+        YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES);
+    if (configuredPolicies == null || configuredPolicies.length == 0) {
+      return;
+    }
+
+    Set<String> configurePoliciesSet = new HashSet<>();
+    for (String s : configuredPolicies) {
+      configurePoliciesSet.add(s);
+    }
+
+    // Add new monitor when needed
+    for (String s : configurePoliciesSet) {
+      if (!runningSchedulingMonitors.containsKey(s)) {
+        Class<?> policyClass;
+        try {
+          policyClass = Class.forName(s);
+        } catch (ClassNotFoundException e) {
+          String message = "Failed to find class of specified policy=" + s;
+          LOG.warn(message);
+          throw new YarnException(message);
+        }
+
+        if (SchedulingEditPolicy.class.isAssignableFrom(policyClass)) {
+          SchedulingEditPolicy policyInstance =
+              (SchedulingEditPolicy) ReflectionUtils.newInstance(policyClass,
+                  null);
+          SchedulingMonitor mon = new SchedulingMonitor(rmContext,
+              policyInstance);
+          mon.init(conf);
+          if (startImmediately) {
+            mon.start();
+          }
+          runningSchedulingMonitors.put(s, mon);
+        } else {
+          String message =
+              "Specified policy=" + s + " is not a SchedulingEditPolicy class.";
+          LOG.warn(message);
+          throw new YarnException(message);
+        }
+      }
+    }
+
+    // Stop monitor when needed.
+    Set<String> disabledPolicies = Sets.difference(
+        runningSchedulingMonitors.keySet(), configurePoliciesSet);
+    for (String disabledPolicy : disabledPolicies) {
+      LOG.info("SchedulingEditPolicy=" + disabledPolicy
+          + " removed, stopping it now ...");
+      silentlyStopSchedulingMonitor(disabledPolicy);
+      runningSchedulingMonitors.remove(disabledPolicy);
+    }
+  }
+
+  public synchronized void initialize(RMContext rmContext,
+      Configuration configuration) throws YarnException {
+    this.rmContext = rmContext;
+    stopAndRemoveAll();
+
+    updateSchedulingMonitors(configuration, false);
+  }
+
+  public synchronized void reinitialize(RMContext rmContext,
+      Configuration configuration) throws YarnException {
+    this.rmContext = rmContext;
+
+    updateSchedulingMonitors(configuration, true);
+  }
+
+  public synchronized void startAll() {
+    for (SchedulingMonitor schedulingMonitor : runningSchedulingMonitors
+        .values()) {
+      schedulingMonitor.start();
+    }
+  }
+
+  private void silentlyStopSchedulingMonitor(String name) {
+    SchedulingMonitor mon = runningSchedulingMonitors.get(name);
+    try {
+      mon.stop();
+      LOG.info("Sucessfully stopped monitor=" + mon.getName());
+    } catch (Exception e) {
+      LOG.warn("Exception while stopping monitor=" + mon.getName(), e);
+    }
+  }
+
+  private void stopAndRemoveAll() {
+    if (!runningSchedulingMonitors.isEmpty()) {
+      for (String schedulingMonitorName : runningSchedulingMonitors
+          .keySet()) {
+        silentlyStopSchedulingMonitor(schedulingMonitorName);
+      }
+      runningSchedulingMonitors.clear();
+    }
+  }
+
+  public boolean isRSMEmpty() {
+    return runningSchedulingMonitors.isEmpty();
+  }
+
+  public boolean isSameConfiguredPolicies(Set<String> configurePoliciesSet) {
+    return configurePoliciesSet.equals(runningSchedulingMonitors.keySet());
+  }
+
+  public SchedulingMonitor getAvailableSchedulingMonitor() {
+    if (isRSMEmpty()) {
+      return null;
+    }
+    for (SchedulingMonitor smon : runningSchedulingMonitors.values()) {
+      if (smon.getSchedulingEditPolicy()
+          instanceof ProportionalCapacityPreemptionPolicy) {
+        return smon;
+      }
+    }
+    return null;
+  }
+
+  public synchronized void stop() throws YarnException {
+    stopAndRemoveAll();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a63d19d3/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 e818dab..4749c3d 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
@@ -68,6 +68,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMCriticalThreadUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitorManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
@@ -168,6 +169,8 @@ public abstract class AbstractYarnScheduler
   // the NM in the next heartbeat.
   private boolean autoUpdateContainers = false;
 
+  protected SchedulingMonitorManager schedulingMonitorManager;
+
   /**
    * Construct the service.
    *
@@ -207,8 +210,8 @@ public abstract class AbstractYarnScheduler
           new RMCriticalThreadUncaughtExceptionHandler(rmContext));
       updateThread.setDaemon(true);
     }
-
     super.serviceInit(conf);
+
   }
 
   @Override
@@ -216,6 +219,7 @@ public abstract class AbstractYarnScheduler
     if (updateThread != null) {
       updateThread.start();
     }
+    schedulingMonitorManager.startAll();
     super.serviceStart();
   }
 
@@ -225,6 +229,9 @@ public abstract class AbstractYarnScheduler
       updateThread.interrupt();
       updateThread.join(THREAD_JOIN_TIMEOUT_MS);
     }
+    if (schedulingMonitorManager != null) {
+      schedulingMonitorManager.stop();
+    }
     super.serviceStop();
   }
 
@@ -233,6 +240,11 @@ public abstract class AbstractYarnScheduler
     return nodeTracker;
   }
 
+  @VisibleForTesting
+  public SchedulingMonitorManager getSchedulingMonitorManager() {
+    return schedulingMonitorManager;
+  }
+
   /*
    * YARN-3136 removed synchronized lock for this method for performance
    * purposes
@@ -1415,4 +1427,15 @@ public abstract class AbstractYarnScheduler
       updateThreadMonitor.notify();
     }
   }
+
+  @Override
+  public void reinitialize(Configuration conf, RMContext rmContext)
+      throws IOException {
+    try {
+      LOG.info("Reinitializing SchedulingMonitorManager ...");
+      schedulingMonitorManager.reinitialize(rmContext, conf);
+    } catch (YarnException e) {
+      throw new IOException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a63d19d3/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 218adf3..de93a6a 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
@@ -62,6 +62,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitorManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementFactory;
@@ -390,6 +391,9 @@ public class CapacityScheduler extends
     Configuration configuration = new Configuration(conf);
     super.serviceInit(conf);
     initScheduler(configuration);
+    // Initialize SchedulingMonitorManager
+    schedulingMonitorManager = new SchedulingMonitorManager();
+    schedulingMonitorManager.initialize(rmContext, conf);
   }
 
   @Override
@@ -444,6 +448,8 @@ public class CapacityScheduler extends
 
       // Setup how many containers we can allocate for each round
       offswitchPerHeartbeatLimit = this.conf.getOffSwitchPerHeartbeatLimit();
+
+      super.reinitialize(newConf, rmContext);
     } finally {
       writeLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a63d19d3/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 625009d..ebc7222 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
@@ -52,6 +52,7 @@ 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.monitor.SchedulingMonitorManager;
 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.rmapp.RMApp;
@@ -1352,6 +1353,10 @@ public class FairScheduler extends
   public void serviceInit(Configuration conf) throws Exception {
     initScheduler(conf);
     super.serviceInit(conf);
+
+    // Initialize SchedulingMonitorManager
+    schedulingMonitorManager = new SchedulingMonitorManager();
+    schedulingMonitorManager.initialize(rmContext, conf);
   }
 
   @Override
@@ -1389,6 +1394,7 @@ public class FairScheduler extends
       throws IOException {
     try {
       allocsLoader.reloadAllocations();
+      super.reinitialize(conf, rmContext);
     } catch (Exception e) {
       LOG.error("Failed to reload allocations file", e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a63d19d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
index 3288912..826575d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitorManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
@@ -255,6 +256,10 @@ public class FifoScheduler extends
   public void serviceInit(Configuration conf) throws Exception {
     initScheduler(conf);
     super.serviceInit(conf);
+
+    // Initialize SchedulingMonitorManager
+    schedulingMonitorManager = new SchedulingMonitorManager();
+    schedulingMonitorManager.initialize(rmContext, conf);
   }
 
   @Override
@@ -312,6 +317,7 @@ public class FifoScheduler extends
       reinitialize(Configuration conf, RMContext rmContext) throws IOException
   {
     setConf(conf);
+    super.reinitialize(conf, rmContext);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a63d19d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.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/RMHATestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java
index 4ac4fc3..439a449 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java
@@ -105,9 +105,35 @@ public abstract class RMHATestBase extends ClientBaseWithFixes{
     return am;
   }
 
+  private MockRM initMockRMWithOldConf(Configuration confForRM1) {
+    return new MockRM(confForRM1, null, false, false) {
+      @Override
+      protected AdminService createAdminService() {
+        return new AdminService(this) {
+          @Override
+          protected void startServer() {
+            // override to not start rpc handler
+          }
+
+          @Override
+          protected void stopServer() {
+            // don't do anything
+          }
+
+          @Override
+          protected Configuration loadNewConfiguration()
+              throws IOException, YarnException {
+            return confForRM1;
+          }
+        };
+      }
+    };
+  }
+
   protected void startRMs() throws IOException {
-    rm1 = new MockRM(confForRM1, null, false, false);
-    rm2 = new MockRM(confForRM2, null, false, false);
+    rm1 = initMockRMWithOldConf(confForRM1);
+    rm2 = initMockRMWithOldConf(confForRM2);
+
     startRMs(rm1, confForRM1, rm2, confForRM2);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a63d19d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/TestSchedulingMonitor.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/monitor/TestSchedulingMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/TestSchedulingMonitor.java
index c38236d..84126c7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/TestSchedulingMonitor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/TestSchedulingMonitor.java
@@ -23,8 +23,15 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.junit.Test;
 
+import java.util.HashSet;
+import java.util.Set;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.timeout;
 import static org.mockito.Mockito.verify;
@@ -51,4 +58,38 @@ public class TestSchedulingMonitor {
     monitor.close();
     rm.close();
   }
+
+  @Test(timeout = 10000)
+  public void testRMUpdateSchedulingEditPolicy() throws Exception {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    conf.setBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true);
+    MockRM rm = new MockRM(conf);
+    rm.start();
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    SchedulingMonitorManager smm = cs.getSchedulingMonitorManager();
+
+    // runningSchedulingMonitors should not be empty when initialize RM
+    // scheduler monitor
+    cs.reinitialize(conf, rm.getRMContext());
+    assertFalse(smm.isRSMEmpty());
+
+    // make sure runningSchedulingPolicies contains all the configured policy
+    // in YARNConfiguration
+    String[] configuredPolicies = conf.getStrings(
+        YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES);
+    Set<String> configurePoliciesSet = new HashSet<>();
+    for (String s : configuredPolicies) {
+      configurePoliciesSet.add(s);
+    }
+    assertTrue(smm.isSameConfiguredPolicies(configurePoliciesSet));
+
+    // disable RM scheduler monitor
+    conf.setBoolean(
+        YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_ENABLE_MONITORS);
+    cs.reinitialize(conf, rm.getRMContext());
+    assertTrue(smm.isRSMEmpty());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a63d19d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.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/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
index 694be09..f0ca466 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitorManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
@@ -48,7 +49,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preempti
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
 import org.apache.hadoop.yarn.util.Clock;
@@ -792,21 +792,23 @@ public class TestProportionalCapacityPreemptionPolicy {
     @SuppressWarnings("resource")
     MockRM rm = new MockRM(conf);
     rm.init(conf);
-    
+
     // ProportionalCapacityPreemptionPolicy should be initialized after
     // CapacityScheduler initialized. We will 
     // 1) find SchedulingMonitor from RMActiveService's service list, 
     // 2) check if ResourceCalculator in policy is null or not. 
     // If it's not null, we can come to a conclusion that policy initialized
     // after scheduler got initialized
-    for (Service service : rm.getRMActiveService().getServices()) {
-      if (service instanceof SchedulingMonitor) {
-        ProportionalCapacityPreemptionPolicy policy =
-            (ProportionalCapacityPreemptionPolicy) ((SchedulingMonitor) service)
-                .getSchedulingEditPolicy();
-        assertNotNull(policy.getResourceCalculator());
-        return;
-      }
+    // Get SchedulingMonitor from SchedulingMonitorManager instead
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    SchedulingMonitorManager smm = cs.getSchedulingMonitorManager();
+    Service service = smm.getAvailableSchedulingMonitor();
+    if (service instanceof SchedulingMonitor) {
+      ProportionalCapacityPreemptionPolicy policy =
+          (ProportionalCapacityPreemptionPolicy) ((SchedulingMonitor) service)
+              .getSchedulingEditPolicy();
+      assertNotNull(policy.getResourceCalculator());
+      return;
     }
     
     fail("Failed to find SchedulingMonitor service, please check what happened");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a63d19d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerLazyPreemption.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/TestCapacitySchedulerLazyPreemption.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/TestCapacitySchedulerLazyPreemption.java
index 4e4e3c2..a4c7d61 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/TestCapacitySchedulerLazyPreemption.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/TestCapacitySchedulerLazyPreemption.java
@@ -26,6 +26,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitorManager;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -126,7 +128,11 @@ public class TestCapacitySchedulerLazyPreemption
             Resources.createResource(1 * GB), 1)), null);
 
     // Get edit policy and do one update
-    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+    SchedulingMonitorManager smm = ((CapacityScheduler) rm1.
+        getResourceScheduler()).getSchedulingMonitorManager();
+    SchedulingMonitor smon = smm.getAvailableSchedulingMonitor();
+    ProportionalCapacityPreemptionPolicy editPolicy =
+        (ProportionalCapacityPreemptionPolicy) smon.getSchedulingEditPolicy();
 
     // Call edit schedule twice, and check if one container from app1 marked
     // to be "killable"
@@ -209,7 +215,11 @@ public class TestCapacitySchedulerLazyPreemption
             Resources.createResource(1 * GB), 1)), null);
 
     // Get edit policy and do one update
-    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+    SchedulingMonitorManager smm = ((CapacityScheduler) rm1.
+        getResourceScheduler()).getSchedulingMonitorManager();
+    SchedulingMonitor smon = smm.getAvailableSchedulingMonitor();
+    ProportionalCapacityPreemptionPolicy editPolicy =
+        (ProportionalCapacityPreemptionPolicy) smon.getSchedulingEditPolicy();
 
     // Call edit schedule twice, and check if one container from app1 marked
     // to be "killable"
@@ -301,7 +311,11 @@ public class TestCapacitySchedulerLazyPreemption
             Resources.createResource(1 * GB), 1, false)), null);
 
     // Get edit policy and do one update
-    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+    SchedulingMonitorManager smm = ((CapacityScheduler) rm1.
+        getResourceScheduler()).getSchedulingMonitorManager();
+    SchedulingMonitor smon = smm.getAvailableSchedulingMonitor();
+    ProportionalCapacityPreemptionPolicy editPolicy =
+        (ProportionalCapacityPreemptionPolicy) smon.getSchedulingEditPolicy();
 
     // Call edit schedule twice, and check if one container from app1 marked
     // to be "killable"
@@ -387,8 +401,11 @@ public class TestCapacitySchedulerLazyPreemption
     am2.allocate("*", 1 * GB, 1, new ArrayList<ContainerId>());
 
     // Get edit policy and do one update
+    SchedulingMonitorManager smm = ((CapacityScheduler) rm1.
+        getResourceScheduler()).getSchedulingMonitorManager();
+    SchedulingMonitor smon = smm.getAvailableSchedulingMonitor();
     ProportionalCapacityPreemptionPolicy editPolicy =
-        (ProportionalCapacityPreemptionPolicy) getSchedulingEditPolicy(rm1);
+        (ProportionalCapacityPreemptionPolicy) smon.getSchedulingEditPolicy();
 
     // Call edit schedule twice, and check if one container from app1 marked
     // to be "killable"
@@ -487,8 +504,11 @@ public class TestCapacitySchedulerLazyPreemption
     am2.allocate("*", 3 * GB, 1, new ArrayList<ContainerId>());
 
     // Get edit policy and do one update
+    SchedulingMonitorManager smm = ((CapacityScheduler) rm1.
+        getResourceScheduler()).getSchedulingMonitorManager();
+    SchedulingMonitor smon = smm.getAvailableSchedulingMonitor();
     ProportionalCapacityPreemptionPolicy editPolicy =
-        (ProportionalCapacityPreemptionPolicy) getSchedulingEditPolicy(rm1);
+        (ProportionalCapacityPreemptionPolicy) smon.getSchedulingEditPolicy();
 
     // Call edit schedule twice, and check if 3 container from app1 marked
     // to be "killable"
@@ -582,7 +602,11 @@ public class TestCapacitySchedulerLazyPreemption
             Resources.createResource(1 * GB), 1)), null);
 
     // Get edit policy and do one update
-    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+    SchedulingMonitorManager smm = ((CapacityScheduler) rm1.
+        getResourceScheduler()).getSchedulingMonitorManager();
+    SchedulingMonitor smon = smm.getAvailableSchedulingMonitor();
+    ProportionalCapacityPreemptionPolicy editPolicy =
+        (ProportionalCapacityPreemptionPolicy) smon.getSchedulingEditPolicy();
 
     // Call edit schedule twice, and check if no container from app1 marked
     // to be "killable"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a63d19d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerSurgicalPreemption.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/TestCapacitySchedulerSurgicalPreemption.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/TestCapacitySchedulerSurgicalPreemption.java
index 9146373..8a7e03f 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/TestCapacitySchedulerSurgicalPreemption.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/TestCapacitySchedulerSurgicalPreemption.java
@@ -26,7 +26,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
-import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
+import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitorManager;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -138,7 +139,11 @@ public class TestCapacitySchedulerSurgicalPreemption
     Assert.assertNotNull(cs.getNode(nm1.getNodeId()).getReservedContainer());
 
     // Get edit policy and do one update
-    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+    SchedulingMonitorManager smm = ((CapacityScheduler) rm1.
+        getResourceScheduler()).getSchedulingMonitorManager();
+    SchedulingMonitor smon = smm.getAvailableSchedulingMonitor();
+    ProportionalCapacityPreemptionPolicy editPolicy =
+        (ProportionalCapacityPreemptionPolicy) smon.getSchedulingEditPolicy();
 
     // Call edit schedule twice, and check if 4 containers from app1 at n1 killed
     editPolicy.editSchedule();
@@ -217,8 +222,11 @@ public class TestCapacitySchedulerSurgicalPreemption
         ApplicationAttemptId.newInstance(app2.getApplicationId(), 1));
 
     // Call editSchedule: containers are selected to be preemption candidate
+    SchedulingMonitorManager smm = ((CapacityScheduler) rm1.
+        getResourceScheduler()).getSchedulingMonitorManager();
+    SchedulingMonitor smon = smm.getAvailableSchedulingMonitor();
     ProportionalCapacityPreemptionPolicy editPolicy =
-        (ProportionalCapacityPreemptionPolicy) getSchedulingEditPolicy(rm1);
+        (ProportionalCapacityPreemptionPolicy) smon.getSchedulingEditPolicy();
     editPolicy.editSchedule();
     Assert.assertEquals(3, editPolicy.getToPreemptContainers().size());
 
@@ -323,8 +331,11 @@ public class TestCapacitySchedulerSurgicalPreemption
     }
 
     // Call editSchedule immediately: containers are not selected
+    SchedulingMonitorManager smm = ((CapacityScheduler) rm1.
+        getResourceScheduler()).getSchedulingMonitorManager();
+    SchedulingMonitor smon = smm.getAvailableSchedulingMonitor();
     ProportionalCapacityPreemptionPolicy editPolicy =
-        (ProportionalCapacityPreemptionPolicy) getSchedulingEditPolicy(rm1);
+        (ProportionalCapacityPreemptionPolicy) smon.getSchedulingEditPolicy();
     editPolicy.editSchedule();
     Assert.assertEquals(0, editPolicy.getToPreemptContainers().size());
 
@@ -434,8 +445,11 @@ public class TestCapacitySchedulerSurgicalPreemption
         cs.getNode(rmNode3.getNodeID()).getReservedContainer());
 
     // Call editSchedule immediately: nothing happens
+    SchedulingMonitorManager smm = ((CapacityScheduler) rm1.
+        getResourceScheduler()).getSchedulingMonitorManager();
+    SchedulingMonitor smon = smm.getAvailableSchedulingMonitor();
     ProportionalCapacityPreemptionPolicy editPolicy =
-        (ProportionalCapacityPreemptionPolicy) getSchedulingEditPolicy(rm1);
+        (ProportionalCapacityPreemptionPolicy) smon.getSchedulingEditPolicy();
     editPolicy.editSchedule();
     Assert.assertNotNull(
         cs.getNode(rmNode3.getNodeID()).getReservedContainer());
@@ -562,8 +576,11 @@ public class TestCapacitySchedulerSurgicalPreemption
     // 6 (selected) + 1 (allocated) which makes target capacity to 70%
     Thread.sleep(1000);
 
+    SchedulingMonitorManager smm = ((CapacityScheduler) rm1.
+        getResourceScheduler()).getSchedulingMonitorManager();
+    SchedulingMonitor smon = smm.getAvailableSchedulingMonitor();
     ProportionalCapacityPreemptionPolicy editPolicy =
-        (ProportionalCapacityPreemptionPolicy) getSchedulingEditPolicy(rm1);
+        (ProportionalCapacityPreemptionPolicy) smon.getSchedulingEditPolicy();
     editPolicy.editSchedule();
     checkNumberOfPreemptionCandidateFromApp(editPolicy, 6,
         am1.getApplicationAttemptId());
@@ -715,8 +732,11 @@ public class TestCapacitySchedulerSurgicalPreemption
     Thread.sleep(1000);
 
     /* 1st container preempted is on n2 */
+    SchedulingMonitorManager smm = ((CapacityScheduler) rm1.
+        getResourceScheduler()).getSchedulingMonitorManager();
+    SchedulingMonitor smon = smm.getAvailableSchedulingMonitor();
     ProportionalCapacityPreemptionPolicy editPolicy =
-        (ProportionalCapacityPreemptionPolicy) getSchedulingEditPolicy(rm1);
+        (ProportionalCapacityPreemptionPolicy) smon.getSchedulingEditPolicy();
     editPolicy.editSchedule();
 
     // We should have one to-preempt container, on node[2]
@@ -887,7 +907,11 @@ public class TestCapacitySchedulerSurgicalPreemption
     waitNumberOfReservedContainersFromApp(schedulerApp2, 1);
 
     // Call editSchedule twice and allocation once, container should get allocated
-    SchedulingEditPolicy editPolicy = getSchedulingEditPolicy(rm1);
+    SchedulingMonitorManager smm = ((CapacityScheduler) rm1.
+        getResourceScheduler()).getSchedulingMonitorManager();
+    SchedulingMonitor smon = smm.getAvailableSchedulingMonitor();
+    ProportionalCapacityPreemptionPolicy editPolicy =
+        (ProportionalCapacityPreemptionPolicy) smon.getSchedulingEditPolicy();
     editPolicy.editSchedule();
     editPolicy.editSchedule();
 


---------------------------------------------------------------------
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: Revert "HDFS-11576. Block recovery will fail indefinitely if recovery time > heartbeat interval. Contributed by Lukas Majercak"

Posted by ae...@apache.org.
Revert "HDFS-11576. Block recovery will fail indefinitely if recovery time > heartbeat interval. Contributed by Lukas Majercak"

This reverts commit 5304698dc8c5667c33e6ed9c4a827ef57172a723.


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

Branch: refs/heads/HDFS-7240
Commit: 53bbef3802194b7a0a3ce5cd3c91def9e88856e3
Parents: 7225ec0
Author: Chris Douglas <cd...@apache.org>
Authored: Fri Dec 1 11:19:01 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 1 11:19:38 2017 -0800

----------------------------------------------------------------------
 .../apache/hadoop/test/GenericTestUtils.java    |  10 +-
 .../server/blockmanagement/BlockManager.java    |  40 ------
 .../blockmanagement/PendingRecoveryBlocks.java  | 143 -------------------
 .../hdfs/server/namenode/FSNamesystem.java      |  40 +++---
 .../TestPendingRecoveryBlocks.java              |  87 -----------
 .../hdfs/server/datanode/TestBlockRecovery.java | 108 --------------
 .../namenode/ha/TestPipelinesFailover.java      |   5 +-
 7 files changed, 20 insertions(+), 413 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/53bbef38/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
index cdde48c..0db6c73 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
@@ -641,16 +641,10 @@ public abstract class GenericTestUtils {
    * conditions.
    */
   public static class SleepAnswer implements Answer<Object> {
-    private final int minSleepTime;
     private final int maxSleepTime;
     private static Random r = new Random();
-
+    
     public SleepAnswer(int maxSleepTime) {
-      this(0, maxSleepTime);
-    }
-
-    public SleepAnswer(int minSleepTime, int maxSleepTime) {
-      this.minSleepTime = minSleepTime;
       this.maxSleepTime = maxSleepTime;
     }
     
@@ -658,7 +652,7 @@ public abstract class GenericTestUtils {
     public Object answer(InvocationOnMock invocation) throws Throwable {
       boolean interrupted = false;
       try {
-        Thread.sleep(r.nextInt(maxSleepTime) + minSleepTime);
+        Thread.sleep(r.nextInt(maxSleepTime));
       } catch (InterruptedException ie) {
         interrupted = true;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53bbef38/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 1cdb159..4986027 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -164,8 +164,6 @@ public class BlockManager implements BlockStatsMXBean {
   private static final String QUEUE_REASON_FUTURE_GENSTAMP =
     "generation stamp is in the future";
 
-  private static final long BLOCK_RECOVERY_TIMEOUT_MULTIPLIER = 30;
-
   private final Namesystem namesystem;
 
   private final BlockManagerSafeMode bmSafeMode;
@@ -355,9 +353,6 @@ public class BlockManager implements BlockStatsMXBean {
   @VisibleForTesting
   final PendingReconstructionBlocks pendingReconstruction;
 
-  /** Stores information about block recovery attempts. */
-  private final PendingRecoveryBlocks pendingRecoveryBlocks;
-
   /** The maximum number of replicas allowed for a block */
   public final short maxReplication;
   /**
@@ -554,12 +549,6 @@ public class BlockManager implements BlockStatsMXBean {
     }
     this.minReplicationToBeInMaintenance = (short)minMaintenanceR;
 
-    long heartbeatIntervalSecs = conf.getTimeDuration(
-        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
-        DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS);
-    long blockRecoveryTimeout = getBlockRecoveryTimeout(heartbeatIntervalSecs);
-    pendingRecoveryBlocks = new PendingRecoveryBlocks(blockRecoveryTimeout);
-
     this.blockReportLeaseManager = new BlockReportLeaseManager(conf);
 
     bmSafeMode = new BlockManagerSafeMode(this, namesystem, haEnabled, conf);
@@ -4747,25 +4736,6 @@ public class BlockManager implements BlockStatsMXBean {
     }
   }
 
-  /**
-   * Notification of a successful block recovery.
-   * @param block for which the recovery succeeded
-   */
-  public void successfulBlockRecovery(BlockInfo block) {
-    pendingRecoveryBlocks.remove(block);
-  }
-
-  /**
-   * Checks whether a recovery attempt has been made for the given block.
-   * If so, checks whether that attempt has timed out.
-   * @param b block for which recovery is being attempted
-   * @return true if no recovery attempt has been made or
-   *         the previous attempt timed out
-   */
-  public boolean addBlockRecoveryAttempt(BlockInfo b) {
-    return pendingRecoveryBlocks.add(b);
-  }
-
   @VisibleForTesting
   public void flushBlockOps() throws IOException {
     runBlockOp(new Callable<Void>(){
@@ -4893,14 +4863,4 @@ public class BlockManager implements BlockStatsMXBean {
     }
     return i;
   }
-
-  private static long getBlockRecoveryTimeout(long heartbeatIntervalSecs) {
-    return TimeUnit.SECONDS.toMillis(heartbeatIntervalSecs *
-        BLOCK_RECOVERY_TIMEOUT_MULTIPLIER);
-  }
-
-  @VisibleForTesting
-  public void setBlockRecoveryTimeout(long blockRecoveryTimeout) {
-    pendingRecoveryBlocks.setRecoveryTimeoutInterval(blockRecoveryTimeout);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53bbef38/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingRecoveryBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingRecoveryBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingRecoveryBlocks.java
deleted file mode 100644
index 3f5f27c..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingRecoveryBlocks.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hdfs.util.LightWeightHashSet;
-import org.apache.hadoop.util.Time;
-import org.slf4j.Logger;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * PendingRecoveryBlocks tracks recovery attempts for each block and their
- * timeouts to ensure we do not have multiple recoveries at the same time
- * and retry only after the timeout for a recovery has expired.
- */
-class PendingRecoveryBlocks {
-  private static final Logger LOG = BlockManager.LOG;
-
-  /** List of recovery attempts per block and the time they expire. */
-  private final LightWeightHashSet<BlockRecoveryAttempt> recoveryTimeouts =
-      new LightWeightHashSet<>();
-
-  /** The timeout for issuing a block recovery again.
-   * (it should be larger than the time to recover a block)
-   */
-  private long recoveryTimeoutInterval;
-
-  PendingRecoveryBlocks(long timeout) {
-    this.recoveryTimeoutInterval = timeout;
-  }
-
-  /**
-   * Remove recovery attempt for the given block.
-   * @param block whose recovery attempt to remove.
-   */
-  synchronized void remove(BlockInfo block) {
-    recoveryTimeouts.remove(new BlockRecoveryAttempt(block));
-  }
-
-  /**
-   * Checks whether a recovery attempt has been made for the given block.
-   * If so, checks whether that attempt has timed out.
-   * @param block block for which recovery is being attempted
-   * @return true if no recovery attempt has been made or
-   *         the previous attempt timed out
-   */
-  synchronized boolean add(BlockInfo block) {
-    boolean added = false;
-    long curTime = getTime();
-    BlockRecoveryAttempt recoveryAttempt =
-        recoveryTimeouts.getElement(new BlockRecoveryAttempt(block));
-
-    if (recoveryAttempt == null) {
-      BlockRecoveryAttempt newAttempt = new BlockRecoveryAttempt(
-          block, curTime + recoveryTimeoutInterval);
-      added = recoveryTimeouts.add(newAttempt);
-    } else if (recoveryAttempt.hasTimedOut(curTime)) {
-      // Previous attempt timed out, reset the timeout
-      recoveryAttempt.setTimeout(curTime + recoveryTimeoutInterval);
-      added = true;
-    } else {
-      long timeoutIn = TimeUnit.MILLISECONDS.toSeconds(
-          recoveryAttempt.timeoutAt - curTime);
-      LOG.info("Block recovery attempt for " + block + " rejected, as the " +
-          "previous attempt times out in " + timeoutIn + " seconds.");
-    }
-    return added;
-  }
-
-  /**
-   * Check whether the given block is under recovery.
-   * @param b block for which to check
-   * @return true if the given block is being recovered
-   */
-  synchronized boolean isUnderRecovery(BlockInfo b) {
-    BlockRecoveryAttempt recoveryAttempt =
-        recoveryTimeouts.getElement(new BlockRecoveryAttempt(b));
-    return recoveryAttempt != null;
-  }
-
-  long getTime() {
-    return Time.monotonicNow();
-  }
-
-  @VisibleForTesting
-  synchronized void setRecoveryTimeoutInterval(long recoveryTimeoutInterval) {
-    this.recoveryTimeoutInterval = recoveryTimeoutInterval;
-  }
-
-  /**
-   * Tracks timeout for block recovery attempt of a given block.
-   */
-  private static class BlockRecoveryAttempt {
-    private final BlockInfo blockInfo;
-    private long timeoutAt;
-
-    private BlockRecoveryAttempt(BlockInfo blockInfo) {
-      this(blockInfo, 0);
-    }
-
-    BlockRecoveryAttempt(BlockInfo blockInfo, long timeoutAt) {
-      this.blockInfo = blockInfo;
-      this.timeoutAt = timeoutAt;
-    }
-
-    boolean hasTimedOut(long currentTime) {
-      return currentTime > timeoutAt;
-    }
-
-    void setTimeout(long newTimeoutAt) {
-      this.timeoutAt = newTimeoutAt;
-    }
-
-    @Override
-    public int hashCode() {
-      return blockInfo.hashCode();
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (obj instanceof BlockRecoveryAttempt) {
-        return this.blockInfo.equals(((BlockRecoveryAttempt) obj).blockInfo);
-      }
-      return false;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53bbef38/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 6a890e2..d3d9cdc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -3318,30 +3318,25 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             + "Removed empty last block and closed file " + src);
         return true;
       }
-      // Start recovery of the last block for this file
-      // Only do so if there is no ongoing recovery for this block,
-      // or the previous recovery for this block timed out.
-      if (blockManager.addBlockRecoveryAttempt(lastBlock)) {
-        long blockRecoveryId = nextGenerationStamp(
-            blockManager.isLegacyBlock(lastBlock));
-        if(copyOnTruncate) {
-          lastBlock.setGenerationStamp(blockRecoveryId);
-        } else if(truncateRecovery) {
-          recoveryBlock.setGenerationStamp(blockRecoveryId);
-        }
-        uc.initializeBlockRecovery(lastBlock, blockRecoveryId, true);
-
-        // Cannot close file right now, since the last block requires recovery.
-        // This may potentially cause infinite loop in lease recovery
-        // if there are no valid replicas on data-nodes.
-        NameNode.stateChangeLog.warn(
-            "DIR* NameSystem.internalReleaseLease: " +
-                "File " + src + " has not been closed." +
-                " Lease recovery is in progress. " +
-                "RecoveryId = " + blockRecoveryId + " for block " + lastBlock);
-      }
+      // start recovery of the last block for this file
+      long blockRecoveryId = nextGenerationStamp(
+          blockManager.isLegacyBlock(lastBlock));
       lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
+      if(copyOnTruncate) {
+        lastBlock.setGenerationStamp(blockRecoveryId);
+      } else if(truncateRecovery) {
+        recoveryBlock.setGenerationStamp(blockRecoveryId);
+      }
+      uc.initializeBlockRecovery(lastBlock, blockRecoveryId, true);
       leaseManager.renewLease(lease);
+      // Cannot close file right now, since the last block requires recovery.
+      // This may potentially cause infinite loop in lease recovery
+      // if there are no valid replicas on data-nodes.
+      NameNode.stateChangeLog.warn(
+                "DIR* NameSystem.internalReleaseLease: " +
+                "File " + src + " has not been closed." +
+               " Lease recovery is in progress. " +
+                "RecoveryId = " + blockRecoveryId + " for block " + lastBlock);
       break;
     }
     return false;
@@ -3609,7 +3604,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         // If this commit does not want to close the file, persist blocks
         FSDirWriteFileOp.persistBlocks(dir, src, iFile, false);
       }
-      blockManager.successfulBlockRecovery(storedBlock);
     } finally {
       writeUnlock("commitBlockSynchronization");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53bbef38/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingRecoveryBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingRecoveryBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingRecoveryBlocks.java
deleted file mode 100644
index baad89f..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingRecoveryBlocks.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-/**
- * This class contains unit tests for PendingRecoveryBlocks.java functionality.
- */
-public class TestPendingRecoveryBlocks {
-
-  private PendingRecoveryBlocks pendingRecoveryBlocks;
-  private final long recoveryTimeout = 1000L;
-
-  private final BlockInfo blk1 = getBlock(1);
-  private final BlockInfo blk2 = getBlock(2);
-  private final BlockInfo blk3 = getBlock(3);
-
-  @Before
-  public void setUp() {
-    pendingRecoveryBlocks =
-        Mockito.spy(new PendingRecoveryBlocks(recoveryTimeout));
-  }
-
-  BlockInfo getBlock(long blockId) {
-    return new BlockInfoContiguous(new Block(blockId), (short) 0);
-  }
-
-  @Test
-  public void testAddDifferentBlocks() {
-    assertTrue(pendingRecoveryBlocks.add(blk1));
-    assertTrue(pendingRecoveryBlocks.isUnderRecovery(blk1));
-    assertTrue(pendingRecoveryBlocks.add(blk2));
-    assertTrue(pendingRecoveryBlocks.isUnderRecovery(blk2));
-    assertTrue(pendingRecoveryBlocks.add(blk3));
-    assertTrue(pendingRecoveryBlocks.isUnderRecovery(blk3));
-  }
-
-  @Test
-  public void testAddAndRemoveBlocks() {
-    // Add blocks
-    assertTrue(pendingRecoveryBlocks.add(blk1));
-    assertTrue(pendingRecoveryBlocks.add(blk2));
-
-    // Remove blk1
-    pendingRecoveryBlocks.remove(blk1);
-
-    // Adding back blk1 should succeed
-    assertTrue(pendingRecoveryBlocks.add(blk1));
-  }
-
-  @Test
-  public void testAddBlockWithPreviousRecoveryTimedOut() {
-    // Add blk
-    Mockito.doReturn(0L).when(pendingRecoveryBlocks).getTime();
-    assertTrue(pendingRecoveryBlocks.add(blk1));
-
-    // Should fail, has not timed out yet
-    Mockito.doReturn(recoveryTimeout / 2).when(pendingRecoveryBlocks).getTime();
-    assertFalse(pendingRecoveryBlocks.add(blk1));
-
-    // Should succeed after timing out
-    Mockito.doReturn(recoveryTimeout * 2).when(pendingRecoveryBlocks).getTime();
-    assertTrue(pendingRecoveryBlocks.add(blk1));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53bbef38/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
index 208447d..311d5a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
@@ -18,10 +18,7 @@
 
 package org.apache.hadoop.hdfs.server.datanode;
 
-import org.apache.hadoop.hdfs.AppendTestUtil;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
-
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
@@ -46,7 +43,6 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Random;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
@@ -98,7 +94,6 @@ import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.test.GenericTestUtils.SleepAnswer;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
@@ -1040,107 +1035,4 @@ public class TestBlockRecovery {
       Assert.fail("Thread failure: " + failureReason);
     }
   }
-
-  /**
-   * Test for block recovery taking longer than the heartbeat interval.
-   */
-  @Test(timeout = 300000L)
-  public void testRecoverySlowerThanHeartbeat() throws Exception {
-    tearDown(); // Stop the Mocked DN started in startup()
-
-    SleepAnswer delayer = new SleepAnswer(3000, 6000);
-    testRecoveryWithDatanodeDelayed(delayer);
-  }
-
-  /**
-   * Test for block recovery timeout. All recovery attempts will be delayed
-   * and the first attempt will be lost to trigger recovery timeout and retry.
-   */
-  @Test(timeout = 300000L)
-  public void testRecoveryTimeout() throws Exception {
-    tearDown(); // Stop the Mocked DN started in startup()
-    final Random r = new Random();
-
-    // Make sure first commitBlockSynchronization call from the DN gets lost
-    // for the recovery timeout to expire and new recovery attempt
-    // to be started.
-    SleepAnswer delayer = new SleepAnswer(3000) {
-      private final AtomicBoolean callRealMethod = new AtomicBoolean();
-
-      @Override
-      public Object answer(InvocationOnMock invocation) throws Throwable {
-        boolean interrupted = false;
-        try {
-          Thread.sleep(r.nextInt(3000) + 6000);
-        } catch (InterruptedException ie) {
-          interrupted = true;
-        }
-        try {
-          if (callRealMethod.get()) {
-            return invocation.callRealMethod();
-          }
-          callRealMethod.set(true);
-          return null;
-        } finally {
-          if (interrupted) {
-            Thread.currentThread().interrupt();
-          }
-        }
-      }
-    };
-    testRecoveryWithDatanodeDelayed(delayer);
-  }
-
-  private void testRecoveryWithDatanodeDelayed(
-      GenericTestUtils.SleepAnswer recoveryDelayer) throws Exception {
-    Configuration configuration = new HdfsConfiguration();
-    configuration.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
-    MiniDFSCluster cluster = null;
-
-    try {
-      cluster = new MiniDFSCluster.Builder(configuration)
-          .numDataNodes(2).build();
-      cluster.waitActive();
-      final FSNamesystem ns = cluster.getNamesystem();
-      final NameNode nn = cluster.getNameNode();
-      final DistributedFileSystem dfs = cluster.getFileSystem();
-      ns.getBlockManager().setBlockRecoveryTimeout(
-          TimeUnit.SECONDS.toMillis(10));
-
-      // Create a file and never close the output stream to trigger recovery
-      FSDataOutputStream out = dfs.create(new Path("/testSlowRecovery"),
-          (short) 2);
-      out.write(AppendTestUtil.randomBytes(0, 4096));
-      out.hsync();
-
-      List<DataNode> dataNodes = cluster.getDataNodes();
-      for (DataNode datanode : dataNodes) {
-        DatanodeProtocolClientSideTranslatorPB nnSpy =
-            InternalDataNodeTestUtils.spyOnBposToNN(datanode, nn);
-
-        Mockito.doAnswer(recoveryDelayer).when(nnSpy).
-            commitBlockSynchronization(
-                Mockito.any(ExtendedBlock.class), Mockito.anyInt(),
-                Mockito.anyLong(), Mockito.anyBoolean(),
-                Mockito.anyBoolean(), Mockito.anyObject(),
-                Mockito.anyObject());
-      }
-
-      // Make sure hard lease expires to trigger replica recovery
-      cluster.setLeasePeriod(100L, 100L);
-
-      // Wait for recovery to succeed
-      GenericTestUtils.waitFor(new Supplier<Boolean>() {
-        @Override
-        public Boolean get() {
-          return ns.getCompleteBlocksTotal() > 0;
-        }
-      }, 300, 300000);
-
-    } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53bbef38/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
index a565578..dc7f47a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
@@ -25,7 +25,6 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Random;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.Log;
@@ -279,14 +278,12 @@ public class TestPipelinesFailover {
     // Disable permissions so that another user can recover the lease.
     conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, false);
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-
+    
     FSDataOutputStream stm = null;
     final MiniDFSCluster cluster = newMiniCluster(conf, 3);
     try {
       cluster.waitActive();
       cluster.transitionToActive(0);
-      cluster.getNamesystem().getBlockManager().setBlockRecoveryTimeout(
-          TimeUnit.SECONDS.toMillis(1));
       Thread.sleep(500);
 
       LOG.info("Starting with NN 0 active");


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