Pārlūkot izejas kodu

Merge branch 'trunk' into HDFS-6581

arp 10 gadi atpakaļ
vecāks
revīzija
889122bd55
63 mainītis faili ar 1648 papildinājumiem un 546 dzēšanām
  1. 4 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 11 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java
  3. 20 10
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
  4. 122 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java
  5. 0 1
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/MiniKMS.java
  6. 21 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  7. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  8. 23 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZone.java
  9. 43 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  10. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  11. 32 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
  12. 110 42
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  13. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  14. 16 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  15. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
  16. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
  17. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/encryption.proto
  18. 21 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
  19. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
  20. 61 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
  21. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataStorage.java
  22. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCacheRevocation.java
  23. 9 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java
  24. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
  25. 7 0
      hadoop-mapreduce-project/CHANGES.txt
  26. 1 62
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
  27. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  28. 11 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
  29. 165 107
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
  30. 52 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/ResourceCalculatorUtils.java
  31. 54 28
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
  32. 57 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
  33. 0 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestUberAM.java
  34. 15 0
      hadoop-yarn-project/CHANGES.txt
  35. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
  36. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
  37. 67 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
  38. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
  39. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
  40. 48 32
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
  41. 21 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
  42. 173 61
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
  43. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
  44. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  45. 20 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java
  46. 114 19
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
  47. 7 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptContainerFinishedEvent.java
  48. 7 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
  49. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java
  50. 41 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeFinishedContainersPulledByAMEvent.java
  51. 39 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
  52. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
  53. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
  54. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java
  55. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
  56. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
  57. 23 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java
  58. 17 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
  59. 7 5
      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
  60. 119 24
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
  61. 16 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
  62. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.java
  63. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm

+ 4 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -847,6 +847,10 @@ Release 2.6.0 - UNRELEASED
 
     HADOOP-10946. Fix a bunch of typos in log messages (Ray Chiang via aw)
 
+    HADOOP-10131. NetWorkTopology#countNumOfAvailableNodes() is returning
+    wrong value if excluded nodes passed are not part of the cluster tree
+    (vinayakumarb)
+
 Release 2.5.1 - 2014-09-05
 
   INCOMPATIBLE CHANGES

+ 11 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java

@@ -34,6 +34,7 @@ public class FileEncryptionInfo {
   private final CipherSuite cipherSuite;
   private final byte[] edek;
   private final byte[] iv;
+  private final String keyName;
   private final String ezKeyVersionName;
 
   /**
@@ -42,14 +43,16 @@ public class FileEncryptionInfo {
    * @param suite CipherSuite used to encrypt the file
    * @param edek encrypted data encryption key (EDEK) of the file
    * @param iv initialization vector (IV) used to encrypt the file
+   * @param keyName name of the key used for the encryption zone
    * @param ezKeyVersionName name of the KeyVersion used to encrypt the
    *                         encrypted data encryption key.
    */
   public FileEncryptionInfo(final CipherSuite suite, final byte[] edek,
-      final byte[] iv, final String ezKeyVersionName) {
+      final byte[] iv, final String keyName, final String ezKeyVersionName) {
     checkNotNull(suite);
     checkNotNull(edek);
     checkNotNull(iv);
+    checkNotNull(keyName);
     checkNotNull(ezKeyVersionName);
     checkArgument(edek.length == suite.getAlgorithmBlockSize(),
         "Unexpected key length");
@@ -58,6 +61,7 @@ public class FileEncryptionInfo {
     this.cipherSuite = suite;
     this.edek = edek;
     this.iv = iv;
+    this.keyName = keyName;
     this.ezKeyVersionName = ezKeyVersionName;
   }
 
@@ -83,6 +87,11 @@ public class FileEncryptionInfo {
     return iv;
   }
 
+  /**
+   * @return name of the encryption zone key.
+   */
+  public String getKeyName() { return keyName; }
+
   /**
    * @return name of the encryption zone KeyVersion used to encrypt the
    * encrypted data encryption key (EDEK).
@@ -95,6 +104,7 @@ public class FileEncryptionInfo {
     builder.append("cipherSuite: " + cipherSuite);
     builder.append(", edek: " + Hex.encodeHexString(edek));
     builder.append(", iv: " + Hex.encodeHexString(iv));
+    builder.append(", keyName: " + keyName);
     builder.append(", ezKeyVersionName: " + ezKeyVersionName);
     builder.append("}");
     return builder.toString();

+ 20 - 10
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java

@@ -782,25 +782,35 @@ public class NetworkTopology {
       scope=scope.substring(1);
     }
     scope = NodeBase.normalize(scope);
-    int count=0; // the number of nodes in both scope & excludedNodes
+    int excludedCountInScope = 0; // the number of nodes in both scope & excludedNodes
+    int excludedCountOffScope = 0; // the number of nodes outside scope & excludedNodes
     netlock.readLock().lock();
     try {
-      for(Node node:excludedNodes) {
-        if ((NodeBase.getPath(node)+NodeBase.PATH_SEPARATOR_STR).
-            startsWith(scope+NodeBase.PATH_SEPARATOR_STR)) {
-          count++;
+      for (Node node : excludedNodes) {
+        node = getNode(NodeBase.getPath(node));
+        if (node == null) {
+          continue;
         }
+        if ((NodeBase.getPath(node) + NodeBase.PATH_SEPARATOR_STR)
+            .startsWith(scope + NodeBase.PATH_SEPARATOR_STR)) {
+          excludedCountInScope++;
+        } else {
+          excludedCountOffScope++;
+        }
+      }
+      Node n = getNode(scope);
+      int scopeNodeCount = 0;
+      if (n != null) {
+        scopeNodeCount++;
       }
-      Node n=getNode(scope);
-      int scopeNodeCount=1;
       if (n instanceof InnerNode) {
         scopeNodeCount=((InnerNode)n).getNumOfLeaves();
       }
       if (isExcluded) {
-        return clusterMap.getNumOfLeaves()-
-          scopeNodeCount-excludedNodes.size()+count;
+        return clusterMap.getNumOfLeaves() - scopeNodeCount
+            - excludedCountOffScope;
       } else {
-        return scopeNodeCount-count;
+        return scopeNodeCount - excludedCountInScope;
       }
     } finally {
       netlock.readLock().unlock();

+ 122 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java

@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.net;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestClusterTopology extends Assert {
+
+  public static class NodeElement implements Node {
+    private String location;
+    private String name;
+    private Node parent;
+    private int level;
+
+    public NodeElement(String name) {
+      this.name = name;
+    }
+
+    @Override
+    public String getNetworkLocation() {
+      return location;
+    }
+
+    @Override
+    public void setNetworkLocation(String location) {
+      this.location = location;
+    }
+
+    @Override
+    public String getName() {
+      return name;
+    }
+
+    @Override
+    public Node getParent() {
+      return parent;
+    }
+
+    @Override
+    public void setParent(Node parent) {
+      this.parent = parent;
+    }
+
+    @Override
+    public int getLevel() {
+      return level;
+    }
+
+    @Override
+    public void setLevel(int i) {
+      this.level = i;
+    }
+
+  }
+
+  /**
+   * Test the count of nodes with exclude list
+   */
+  @Test
+  public void testCountNumNodes() throws Exception {
+    // create the topology
+    NetworkTopology cluster = new NetworkTopology();
+    cluster.add(getNewNode("node1", "/d1/r1"));
+    NodeElement node2 = getNewNode("node2", "/d1/r2");
+    cluster.add(node2);
+    cluster.add(getNewNode("node3", "/d1/r3"));
+    NodeElement node3 = getNewNode("node4", "/d1/r4");
+    cluster.add(node3);
+    // create exclude list
+    List<Node> excludedNodes = new ArrayList<Node>();
+
+    assertEquals("4 nodes should be available", 4,
+        cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes));
+    NodeElement deadNode = getNewNode("node5", "/d1/r2");
+    excludedNodes.add(deadNode);
+    assertEquals("4 nodes should be available with extra excluded Node", 4,
+        cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes));
+    // add one existing node to exclude list
+    excludedNodes.add(node3);
+    assertEquals("excluded nodes with ROOT scope should be considered", 3,
+        cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes));
+    assertEquals("excluded nodes without ~ scope should be considered", 2,
+        cluster.countNumOfAvailableNodes("~" + deadNode.getNetworkLocation(),
+            excludedNodes));
+    assertEquals("excluded nodes with rack scope should be considered", 1,
+        cluster.countNumOfAvailableNodes(deadNode.getNetworkLocation(),
+            excludedNodes));
+    // adding the node in excluded scope to excluded list
+    excludedNodes.add(node2);
+    assertEquals("excluded nodes with ~ scope should be considered", 2,
+        cluster.countNumOfAvailableNodes("~" + deadNode.getNetworkLocation(),
+            excludedNodes));
+    // getting count with non-exist scope.
+    assertEquals("No nodes should be considered for non-exist scope", 0,
+        cluster.countNumOfAvailableNodes("/non-exist", excludedNodes));
+  }
+
+  private NodeElement getNewNode(String name, String rackLocation) {
+    NodeElement node = new NodeElement(name);
+    node.setNetworkLocation(rackLocation);
+    return node;
+  }
+}

+ 0 - 1
hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/MiniKMS.java

@@ -169,7 +169,6 @@ public class MiniKMS {
       kms.set(KMSConfiguration.KEY_PROVIDER_URI,
           "jceks://file@" + new Path(kmsConfDir, "kms.keystore").toUri());
       kms.set("hadoop.kms.authentication.type", "simple");
-      kms.setBoolean(KMSConfiguration.KEY_AUTHORIZATION_ENABLE, false);
       Writer writer = new FileWriter(kmsFile);
       kms.writeXml(writer);
       writer.close();

+ 21 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -133,6 +133,9 @@ Trunk (Unreleased)
     HDFS-6609. Use DirectorySnapshottableFeature to represent a snapshottable
     directory. (Jing Zhao via wheat9)
 
+    HDFS-6987. Move CipherSuite xattr information up to the encryption zone
+    root. (Zhe Zhang via wang)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -786,6 +789,12 @@ Release 2.6.0 - UNRELEASED
 
     HDFS-7046. HA NN can NPE upon transition to active. (kihwal)
 
+    HDFS-7106. Reconfiguring DataNode volumes does not release the lock files
+    in removed volumes. (cnauroth via cmccabe)
+
+    HDFS-7001. Tests in TestTracing depends on the order of execution
+    (iwasakims via cmccabe)
+
     BREAKDOWN OF HDFS-6134 AND HADOOP-10150 SUBTASKS AND RELATED JIRAS
   
       HDFS-6387. HDFS CLI admin tool for creating & deleting an
@@ -905,6 +914,18 @@ Release 2.6.0 - UNRELEASED
     HDFS-7105. Allow falling back to a non-SASL connection on
     DataTransferProtocol in several edge cases. (cnauroth)
 
+    HDFS-7107. Avoid Findbugs warning for synchronization on
+    AbstractNNFailoverProxyProvider#fallbackToSimpleAuth. (cnauroth)
+
+    HDFS-7109. TestDataStorage does not release file locks between tests.
+    (cnauroth)
+
+    HDFS-7110. Skip tests related to short-circuit read on platforms that do not
+    currently implement short-circuit read. (cnauroth)
+
+    HDFS-7115. TestEncryptionZones assumes Unix path separator for KMS key store
+    path. (Xiaoyu Yao via cnauroth)
+
 Release 2.5.1 - 2014-09-05
 
   INCOMPATIBLE CHANGES

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -1319,8 +1319,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
           " an encrypted file");
     }
     EncryptedKeyVersion ekv = EncryptedKeyVersion.createForDecryption(
-        //TODO: here we have to put the keyName to be provided by HDFS-6987
-        null, feInfo.getEzKeyVersionName(), feInfo.getIV(),
+        feInfo.getKeyName(), feInfo.getEzKeyVersionName(), feInfo.getIV(),
         feInfo.getEncryptedDataEncryptionKey());
     try {
       return provider.decryptEncryptedKey(ekv);

+ 23 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZone.java

@@ -21,6 +21,7 @@ import org.apache.commons.lang.builder.EqualsBuilder;
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.crypto.CipherSuite;
 
 /**
  * A simple class for representing an encryption zone. Presently an encryption
@@ -31,32 +32,40 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceStability.Evolving
 public class EncryptionZone {
 
+  private final long id;
   private final String path;
+  private final CipherSuite suite;
   private final String keyName;
-  private final long id;
 
-  public EncryptionZone(String path, String keyName, long id) {
+  public EncryptionZone(long id, String path,
+      CipherSuite suite, String keyName) {
+    this.id = id;
     this.path = path;
+    this.suite = suite;
     this.keyName = keyName;
-    this.id = id;
+  }
+
+  public long getId() {
+    return id;
   }
 
   public String getPath() {
     return path;
   }
 
-  public String getKeyName() {
-    return keyName;
+  public CipherSuite getSuite() {
+    return suite;
   }
 
-  public long getId() {
-    return id;
+  public String getKeyName() {
+    return keyName;
   }
 
   @Override
   public int hashCode() {
     return new HashCodeBuilder(13, 31).
-      append(path).append(keyName).append(id).
+      append(id).append(path).
+      append(suite).append(keyName).
       toHashCode();
   }
 
@@ -74,16 +83,18 @@ public class EncryptionZone {
 
     EncryptionZone rhs = (EncryptionZone) obj;
     return new EqualsBuilder().
+      append(id, rhs.id).
       append(path, rhs.path).
+      append(suite, rhs.suite).
       append(keyName, rhs.keyName).
-      append(id, rhs.id).
       isEquals();
   }
 
   @Override
   public String toString() {
-    return "EncryptionZone [path=" + path +
-        ", keyName=" + keyName +
-        ", id=" + id + "]";
+    return "EncryptionZone [id=" + id +
+        ", path=" + path +
+        ", suite=" + suite +
+        ", keyName=" + keyName + "]";
   }
 }

+ 43 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -177,7 +177,6 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
-import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
@@ -2345,12 +2344,14 @@ public class PBHelper {
     return EncryptionZoneProto.newBuilder()
         .setId(zone.getId())
         .setKeyName(zone.getKeyName())
-        .setPath(zone.getPath()).build();
+        .setPath(zone.getPath())
+        .setSuite(convert(zone.getSuite()))
+        .build();
   }
 
   public static EncryptionZone convert(EncryptionZoneProto proto) {
-    return new EncryptionZone(proto.getPath(), proto.getKeyName(),
-        proto.getId());
+    return new EncryptionZone(proto.getId(), proto.getPath(),
+        convert(proto.getSuite()), proto.getKeyName());
   }
 
   public static ShortCircuitShmSlotProto convert(SlotId slotId) {
@@ -2672,12 +2673,36 @@ public class PBHelper {
     }
     return HdfsProtos.FileEncryptionInfoProto.newBuilder()
         .setSuite(convert(info.getCipherSuite()))
+        .setKey(getByteString(info.getEncryptedDataEncryptionKey()))
+        .setIv(getByteString(info.getIV()))
+        .setEzKeyVersionName(info.getEzKeyVersionName())
+        .setKeyName(info.getKeyName())
+        .build();
+  }
+
+  public static HdfsProtos.PerFileEncryptionInfoProto convertPerFileEncInfo(
+      FileEncryptionInfo info) {
+    if (info == null) {
+      return null;
+    }
+    return HdfsProtos.PerFileEncryptionInfoProto.newBuilder()
         .setKey(getByteString(info.getEncryptedDataEncryptionKey()))
         .setIv(getByteString(info.getIV()))
         .setEzKeyVersionName(info.getEzKeyVersionName())
         .build();
   }
 
+  public static HdfsProtos.ZoneEncryptionInfoProto convert(
+      CipherSuite suite, String keyName) {
+    if (suite == null || keyName == null) {
+      return null;
+    }
+    return HdfsProtos.ZoneEncryptionInfoProto.newBuilder()
+        .setSuite(convert(suite))
+        .setKeyName(keyName)
+        .build();
+  }
+
   public static FileEncryptionInfo convert(
       HdfsProtos.FileEncryptionInfoProto proto) {
     if (proto == null) {
@@ -2687,7 +2712,20 @@ public class PBHelper {
     byte[] key = proto.getKey().toByteArray();
     byte[] iv = proto.getIv().toByteArray();
     String ezKeyVersionName = proto.getEzKeyVersionName();
-    return new FileEncryptionInfo(suite, key, iv, ezKeyVersionName);
+    String keyName = proto.getKeyName();
+    return new FileEncryptionInfo(suite, key, iv, keyName, ezKeyVersionName);
+  }
+
+  public static FileEncryptionInfo convert(
+      HdfsProtos.PerFileEncryptionInfoProto fileProto,
+      CipherSuite suite, String keyName) {
+    if (fileProto == null || suite == null || keyName == null) {
+      return null;
+    }
+    byte[] key = fileProto.getKey().toByteArray();
+    byte[] iv = fileProto.getIv().toByteArray();
+    String ezKeyVersionName = fileProto.getEzKeyVersionName();
+    return new FileEncryptionInfo(suite, key, iv, keyName, ezKeyVersionName);
   }
 
 }

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -376,6 +376,13 @@ public class DataStorage extends Storage {
       StorageDirectory sd = it.next();
       if (dataDirs.contains(sd.getRoot())) {
         it.remove();
+        try {
+          sd.unlock();
+        } catch (IOException e) {
+          LOG.warn(String.format(
+            "I/O error attempting to unlock storage directory %s.",
+            sd.getRoot()), e);
+        }
       }
     }
   }

+ 32 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java

@@ -26,6 +26,7 @@ import java.util.TreeMap;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
@@ -33,6 +34,8 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -53,8 +56,8 @@ public class EncryptionZoneManager {
   public static Logger LOG = LoggerFactory.getLogger(EncryptionZoneManager
       .class);
 
-  private static final EncryptionZone NULL_EZ =
-      new EncryptionZone("", "", -1);
+  public static final EncryptionZone NULL_EZ =
+      new EncryptionZone(-1, "", CipherSuite.UNKNOWN, "");
 
   /**
    * EncryptionZoneInt is the internal representation of an encryption zone. The
@@ -62,21 +65,27 @@ public class EncryptionZoneManager {
    * contains the EZ's pathname.
    */
   private static class EncryptionZoneInt {
-    private final String keyName;
     private final long inodeId;
+    private final CipherSuite suite;
+    private final String keyName;
 
-    EncryptionZoneInt(long inodeId, String keyName) {
-      this.keyName = keyName;
+    EncryptionZoneInt(long inodeId, CipherSuite suite, String keyName) {
       this.inodeId = inodeId;
-    }
-
-    String getKeyName() {
-      return keyName;
+      this.suite = suite;
+      this.keyName = keyName;
     }
 
     long getINodeId() {
       return inodeId;
     }
+
+    CipherSuite getSuite() {
+      return suite;
+    }
+
+    String getKeyName() {
+      return keyName;
+    }
   }
 
   private final TreeMap<Long, EncryptionZoneInt> encryptionZones;
@@ -109,9 +118,9 @@ public class EncryptionZoneManager {
    * @param inodeId of the encryption zone
    * @param keyName encryption zone key name
    */
-  void addEncryptionZone(Long inodeId, String keyName) {
+  void addEncryptionZone(Long inodeId, CipherSuite suite, String keyName) {
     assert dir.hasWriteLock();
-    unprotectedAddEncryptionZone(inodeId, keyName);
+    unprotectedAddEncryptionZone(inodeId, suite, keyName);
   }
 
   /**
@@ -122,8 +131,10 @@ public class EncryptionZoneManager {
    * @param inodeId of the encryption zone
    * @param keyName encryption zone key name
    */
-  void unprotectedAddEncryptionZone(Long inodeId, String keyName) {
-    final EncryptionZoneInt ez = new EncryptionZoneInt(inodeId, keyName);
+  void unprotectedAddEncryptionZone(Long inodeId,
+      CipherSuite suite, String keyName) {
+    final EncryptionZoneInt ez = new EncryptionZoneInt(
+        inodeId, suite, keyName);
     encryptionZones.put(inodeId, ez);
   }
 
@@ -207,8 +218,8 @@ public class EncryptionZoneManager {
     if (ezi == null) {
       return NULL_EZ;
     } else {
-      return new EncryptionZone(getFullPathName(ezi), ezi.getKeyName(),
-          ezi.getINodeId());
+      return new EncryptionZone(ezi.getINodeId(), getFullPathName(ezi),
+          ezi.getSuite(), ezi.getKeyName());
     }
   }
 
@@ -264,7 +275,7 @@ public class EncryptionZoneManager {
    * <p/>
    * Called while holding the FSDirectory lock.
    */
-  XAttr createEncryptionZone(String src, String keyName)
+  XAttr createEncryptionZone(String src, CipherSuite suite, String keyName)
       throws IOException {
     assert dir.hasWriteLock();
     if (dir.isNonEmptyDirectory(src)) {
@@ -284,8 +295,10 @@ public class EncryptionZoneManager {
           "encryption zone. (" + getFullPathName(ezi) + ")");
     }
 
+    final HdfsProtos.ZoneEncryptionInfoProto proto =
+        PBHelper.convert(suite, keyName);
     final XAttr ezXAttr = XAttrHelper
-        .buildXAttr(CRYPTO_XATTR_ENCRYPTION_ZONE, keyName.getBytes());
+        .buildXAttr(CRYPTO_XATTR_ENCRYPTION_ZONE, proto.toByteArray());
 
     final List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
     xattrs.add(ezXAttr);
@@ -327,8 +340,8 @@ public class EncryptionZoneManager {
         continue;
       }
       // Add the EZ to the result list
-      zones.add(new EncryptionZone(pathName,
-          ezi.getKeyName(), ezi.getINodeId()));
+      zones.add(new EncryptionZone(ezi.getINodeId(), pathName,
+          ezi.getSuite(), ezi.getKeyName()));
       count++;
       if (count >= numResponses) {
         break;

+ 110 - 42
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -37,6 +37,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
 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.fs.ContentSummary;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileEncryptionInfo;
@@ -1407,9 +1408,10 @@ public class FSDirectory implements Closeable {
       if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
         return getSnapshotsListing(srcs, startAfter);
       }
-      final INodesInPath inodesInPath = getLastINodeInPath(srcs);
+      final INodesInPath inodesInPath = getINodesInPath(srcs, true);
+      final INode[] inodes = inodesInPath.getINodes();
       final int snapshot = inodesInPath.getPathSnapshotId();
-      final INode targetNode = inodesInPath.getLastINode();
+      final INode targetNode = inodes[inodes.length - 1];
       if (targetNode == null)
         return null;
       byte parentStoragePolicy = isSuperUser ?
@@ -1419,7 +1421,7 @@ public class FSDirectory implements Closeable {
         return new DirectoryListing(
             new HdfsFileStatus[]{createFileStatus(HdfsFileStatus.EMPTY_NAME,
                 targetNode, needLocation, parentStoragePolicy, snapshot,
-                isRawPath)}, 0);
+                isRawPath, inodesInPath)}, 0);
       }
 
       final INodeDirectory dirInode = targetNode.asDirectory();
@@ -1436,7 +1438,7 @@ public class FSDirectory implements Closeable {
             cur.getLocalStoragePolicyID(): BlockStoragePolicy.ID_UNSPECIFIED;
         listing[i] = createFileStatus(cur.getLocalNameBytes(), cur, needLocation,
             getStoragePolicyID(curPolicy, parentStoragePolicy), snapshot,
-            isRawPath);
+            isRawPath, inodesInPath);
         listingCnt++;
         if (needLocation) {
             // Once we  hit lsLimit locations, stop.
@@ -1487,7 +1489,8 @@ public class FSDirectory implements Closeable {
     for (int i = 0; i < numOfListing; i++) {
       Root sRoot = snapshots.get(i + skipSize).getRoot();
       listing[i] = createFileStatus(sRoot.getLocalNameBytes(), sRoot,
-          BlockStoragePolicy.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID, false);
+          BlockStoragePolicy.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
+          false, null);
     }
     return new DirectoryListing(
         listing, snapshots.size() - skipSize - numOfListing);
@@ -1510,12 +1513,14 @@ public class FSDirectory implements Closeable {
       if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
         return getFileInfo4DotSnapshot(srcs);
       }
-      final INodesInPath inodesInPath = getLastINodeInPath(srcs, resolveLink);
-      final INode i = inodesInPath.getINode(0);
+      final INodesInPath inodesInPath = getINodesInPath(srcs, resolveLink);
+      final INode[] inodes = inodesInPath.getINodes();
+      final INode i = inodes[inodes.length - 1];
       byte policyId = includeStoragePolicy && i != null && !i.isSymlink() ?
           i.getStoragePolicyID() : BlockStoragePolicy.ID_UNSPECIFIED;
       return i == null ? null : createFileStatus(HdfsFileStatus.EMPTY_NAME, i,
-          policyId, inodesInPath.getPathSnapshotId(), isRawPath);
+          policyId, inodesInPath.getPathSnapshotId(), isRawPath,
+          inodesInPath);
     } finally {
       readUnlock();
     }
@@ -2168,8 +2173,17 @@ public class FSDirectory implements Closeable {
           for (XAttr xattr : xattrs) {
             final String xaName = XAttrHelper.getPrefixName(xattr);
             if (CRYPTO_XATTR_ENCRYPTION_ZONE.equals(xaName)) {
-              ezManager.unprotectedAddEncryptionZone(inode.getId(),
-                  new String(xattr.getValue()));
+              try {
+                final HdfsProtos.ZoneEncryptionInfoProto ezProto =
+                    HdfsProtos.ZoneEncryptionInfoProto.parseFrom(
+                        xattr.getValue());
+                ezManager.unprotectedAddEncryptionZone(inode.getId(),
+                    PBHelper.convert(ezProto.getSuite()),
+                    ezProto.getKeyName());
+              } catch (InvalidProtocolBufferException e) {
+                NameNode.LOG.warn("Error parsing protocol buffer of " +
+                    "EZ XAttr " + xattr.getName());
+              }
             }
           }
         }
@@ -2361,12 +2375,15 @@ public class FSDirectory implements Closeable {
    * @throws IOException if any error occurs
    */
   private HdfsFileStatus createFileStatus(byte[] path, INode node,
-      boolean needLocation, byte storagePolicy, int snapshot, boolean isRawPath)
+      boolean needLocation, byte storagePolicy, int snapshot,
+      boolean isRawPath, INodesInPath iip)
       throws IOException {
     if (needLocation) {
-      return createLocatedFileStatus(path, node, storagePolicy, snapshot, isRawPath);
+      return createLocatedFileStatus(path, node, storagePolicy, snapshot,
+          isRawPath, iip);
     } else {
-      return createFileStatus(path, node, storagePolicy, snapshot, isRawPath);
+      return createFileStatus(path, node, storagePolicy, snapshot,
+          isRawPath, iip);
     }
   }
 
@@ -2374,7 +2391,7 @@ public class FSDirectory implements Closeable {
    * Create FileStatus by file INode 
    */
   HdfsFileStatus createFileStatus(byte[] path, INode node, byte storagePolicy,
-      int snapshot, boolean isRawPath) throws IOException {
+      int snapshot, boolean isRawPath, INodesInPath iip) throws IOException {
      long size = 0;     // length is zero for directories
      short replication = 0;
      long blocksize = 0;
@@ -2382,7 +2399,7 @@ public class FSDirectory implements Closeable {
      final boolean isEncrypted;
 
      final FileEncryptionInfo feInfo = isRawPath ? null :
-         getFileEncryptionInfo(node, snapshot);
+         getFileEncryptionInfo(node, snapshot, iip);
 
      if (node.isFile()) {
        final INodeFile fileNode = node.asFile();
@@ -2422,7 +2439,8 @@ public class FSDirectory implements Closeable {
    * Create FileStatus with location info by file INode
    */
   private HdfsLocatedFileStatus createLocatedFileStatus(byte[] path, INode node,
-      byte storagePolicy, int snapshot, boolean isRawPath) throws IOException {
+      byte storagePolicy, int snapshot, boolean isRawPath,
+      INodesInPath iip) throws IOException {
     assert hasReadLock();
     long size = 0; // length is zero for directories
     short replication = 0;
@@ -2431,7 +2449,7 @@ public class FSDirectory implements Closeable {
     LocatedBlocks loc = null;
     final boolean isEncrypted;
     final FileEncryptionInfo feInfo = isRawPath ? null :
-        getFileEncryptionInfo(node, snapshot);
+        getFileEncryptionInfo(node, snapshot, iip);
     if (node.isFile()) {
       final INodeFile fileNode = node.asFile();
       size = fileNode.computeFileSize(snapshot);
@@ -2756,11 +2774,11 @@ public class FSDirectory implements Closeable {
     }
   }
 
-  XAttr createEncryptionZone(String src, String keyName)
+  XAttr createEncryptionZone(String src, CipherSuite suite, String keyName)
     throws IOException {
     writeLock();
     try {
-      return ezManager.createEncryptionZone(src, keyName);
+      return ezManager.createEncryptionZone(src, suite, keyName);
     } finally {
       writeUnlock();
     }
@@ -2791,7 +2809,8 @@ public class FSDirectory implements Closeable {
   void setFileEncryptionInfo(String src, FileEncryptionInfo info)
       throws IOException {
     // Make the PB for the xattr
-    final HdfsProtos.FileEncryptionInfoProto proto = PBHelper.convert(info);
+    final HdfsProtos.PerFileEncryptionInfoProto proto =
+        PBHelper.convertPerFileEncInfo(info);
     final byte[] protoBytes = proto.toByteArray();
     final XAttr fileEncryptionAttr =
         XAttrHelper.buildXAttr(CRYPTO_XATTR_FILE_ENCRYPTION_INFO, protoBytes);
@@ -2807,35 +2826,64 @@ public class FSDirectory implements Closeable {
   }
 
   /**
-   * Return the FileEncryptionInfo for an INode, or null if the INode is not
-   * an encrypted file.
-   */
-  FileEncryptionInfo getFileEncryptionInfo(INode inode, int snapshotId)
-      throws IOException {
+   * This function combines the per-file encryption info (obtained
+   * from the inode's XAttrs), and the encryption info from its zone, and
+   * returns a consolidated FileEncryptionInfo instance. Null is returned
+   * for non-encrypted files.
+   *
+   * @param inode inode of the file
+   * @param snapshotId ID of the snapshot that
+   *                   we want to get encryption info from
+   * @param iip inodes in the path containing the file, passed in to
+   *            avoid obtaining the list of inodes again; if iip is
+   *            null then the list of inodes will be obtained again
+   * @return consolidated file encryption info; null for non-encrypted files
+   */
+  FileEncryptionInfo getFileEncryptionInfo(INode inode, int snapshotId,
+      INodesInPath iip) throws IOException {
     if (!inode.isFile()) {
       return null;
     }
     readLock();
     try {
-      List<XAttr> xAttrs = XAttrStorage.readINodeXAttrs(inode, snapshotId);
-      if (xAttrs == null) {
-        return null;
+      if (iip == null) {
+        iip = getINodesInPath(inode.getFullPathName(), true);
       }
-      for (XAttr x : xAttrs) {
-        if (XAttrHelper.getPrefixName(x)
-            .equals(CRYPTO_XATTR_FILE_ENCRYPTION_INFO)) {
-          try {
-            HdfsProtos.FileEncryptionInfoProto proto =
-                HdfsProtos.FileEncryptionInfoProto.parseFrom(x.getValue());
-            FileEncryptionInfo feInfo = PBHelper.convert(proto);
-            return feInfo;
-          } catch (InvalidProtocolBufferException e) {
-            throw new IOException("Could not parse file encryption info for " +
-                "inode " + inode, e);
-          }
+      EncryptionZone encryptionZone = getEZForPath(iip);
+      if (encryptionZone == null ||
+          encryptionZone.equals(EncryptionZoneManager.NULL_EZ)) {
+        // not an encrypted file
+        return null;
+      } else if(encryptionZone.getPath() == null
+          || encryptionZone.getPath().isEmpty()) {
+        if (NameNode.LOG.isDebugEnabled()) {
+          NameNode.LOG.debug("Encryption zone " +
+              encryptionZone.getPath() + " does not have a valid path.");
         }
       }
-      return null;
+
+      CipherSuite suite = encryptionZone.getSuite();
+      String keyName = encryptionZone.getKeyName();
+
+      XAttr fileXAttr = unprotectedGetXAttrByName(inode, snapshotId,
+          CRYPTO_XATTR_FILE_ENCRYPTION_INFO);
+
+      if (fileXAttr == null) {
+        NameNode.LOG.warn("Could not find encryption XAttr for file " +
+            inode.getFullPathName() + " in encryption zone " +
+            encryptionZone.getPath());
+        return null;
+      }
+
+      try {
+        HdfsProtos.PerFileEncryptionInfoProto fileProto =
+            HdfsProtos.PerFileEncryptionInfoProto.parseFrom(
+                fileXAttr.getValue());
+        return PBHelper.convert(fileProto, suite, keyName);
+      } catch (InvalidProtocolBufferException e) {
+        throw new IOException("Could not parse file encryption info for " +
+            "inode " + inode, e);
+      }
     } finally {
       readUnlock();
     }
@@ -2870,7 +2918,11 @@ public class FSDirectory implements Closeable {
        * of encryption zones.
        */
       if (CRYPTO_XATTR_ENCRYPTION_ZONE.equals(xaName)) {
-        ezManager.addEncryptionZone(inode.getId(), new String(xattr.getValue()));
+        final HdfsProtos.ZoneEncryptionInfoProto ezProto =
+            HdfsProtos.ZoneEncryptionInfoProto.parseFrom(xattr.getValue());
+        ezManager.addEncryptionZone(inode.getId(),
+            PBHelper.convert(ezProto.getSuite()),
+            ezProto.getKeyName());
       }
 
       if (!isFile && SECURITY_XATTR_UNREADABLE_BY_SUPERUSER.equals(xaName)) {
@@ -2987,6 +3039,22 @@ public class FSDirectory implements Closeable {
     return XAttrStorage.readINodeXAttrs(inode, snapshotId);
   }
 
+  private XAttr unprotectedGetXAttrByName(INode inode, int snapshotId,
+      String xAttrName)
+      throws IOException {
+    List<XAttr> xAttrs = XAttrStorage.readINodeXAttrs(inode, snapshotId);
+    if (xAttrs == null) {
+      return null;
+    }
+    for (XAttr x : xAttrs) {
+      if (XAttrHelper.getPrefixName(x)
+          .equals(xAttrName)) {
+        return x;
+      }
+    }
+    return null;
+  }
+
   private static INode resolveLastINode(String src, INodesInPath iip)
       throws FileNotFoundException {
     INode inode = iip.getLastINode();

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -341,8 +341,10 @@ public class FSEditLogLoader {
       // 3. OP_ADD to open file for append
 
       // See if the file already exists (persistBlocks call)
-      final INodesInPath iip = fsDir.getLastINodeInPath(path);
-      INodeFile oldFile = INodeFile.valueOf(iip.getINode(0), path, true);
+      final INodesInPath iip = fsDir.getINodesInPath(path, true);
+      final INode[] inodes = iip.getINodes();
+      INodeFile oldFile = INodeFile.valueOf(
+          inodes[inodes.length - 1], path, true);
       if (oldFile != null && addCloseOp.overwrite) {
         // This is OP_ADD with overwrite
         fsDir.unprotectedDelete(path, addCloseOp.mtime);
@@ -373,7 +375,7 @@ public class FSEditLogLoader {
           HdfsFileStatus stat = fsNamesys.dir.createFileStatus(
               HdfsFileStatus.EMPTY_NAME, newFile,
               BlockStoragePolicy.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
-              false);
+              false, iip);
           fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
               addCloseOp.rpcCallId, stat);
         }

+ 16 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
 import static org.apache.hadoop.crypto.key.KeyProviderCryptoExtension
     .EncryptedKeyVersion;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT;
@@ -136,6 +135,7 @@ 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.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
 import org.apache.hadoop.fs.CacheFlag;
@@ -167,7 +167,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.UnknownCipherSuiteException;
 import org.apache.hadoop.hdfs.protocol.AclException;
@@ -1865,8 +1864,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           doAccessTime = false;
         }
 
-        final INodesInPath iip = dir.getLastINodeInPath(src);
-        final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src);
+        final INodesInPath iip = dir.getINodesInPath(src, true);
+        final INode[] inodes = iip.getINodes();
+        final INodeFile inode = INodeFile.valueOf(
+            inodes[inodes.length - 1], src);
         if (isPermissionEnabled) {
           checkUnreadableBySuperuser(pc, inode, iip.getPathSnapshotId());
         }
@@ -1899,7 +1900,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
         final FileEncryptionInfo feInfo =
           FSDirectory.isReservedRawName(srcArg) ?
-          null : dir.getFileEncryptionInfo(inode, iip.getPathSnapshotId());
+          null : dir.getFileEncryptionInfo(inode, iip.getPathSnapshotId(),
+              iip);
 
         final LocatedBlocks blocks =
           blockManager.createLocatedBlocks(inode.getBlocks(), fileSize,
@@ -2633,7 +2635,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       feInfo = new FileEncryptionInfo(suite,
           edek.getEncryptedKeyVersion().getMaterial(),
           edek.getEncryptedKeyIv(),
-          edek.getEncryptionKeyVersionName());
+          ezKeyName, edek.getEncryptionKeyVersionName());
       Preconditions.checkNotNull(feInfo);
     }
 
@@ -8723,8 +8725,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         throw new IOException("Must specify a key name when creating an " +
             "encryption zone");
       }
-      KeyVersion keyVersion = provider.getCurrentKey(keyName);
-      if (keyVersion == null) {
+      KeyProvider.Metadata metadata = provider.getMetadata(keyName);
+      if (metadata == null) {
         /*
          * It would be nice if we threw something more specific than
          * IOException when the key is not found, but the KeyProvider API
@@ -8735,7 +8737,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
          */
         throw new IOException("Key " + keyName + " doesn't exist.");
       }
-      createEncryptionZoneInt(src, keyName, cacheEntry != null);
+      createEncryptionZoneInt(src, metadata.getCipher(),
+          keyName, cacheEntry != null);
       success = true;
     } catch (AccessControlException e) {
       logAuditEvent(false, "createEncryptionZone", src);
@@ -8745,8 +8748,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
   }
 
-  private void createEncryptionZoneInt(final String srcArg, String keyName,
-      final boolean logRetryCache) throws IOException {
+  private void createEncryptionZoneInt(final String srcArg, String cipher,
+      String keyName, final boolean logRetryCache) throws IOException {
     String src = srcArg;
     HdfsFileStatus resultingStat = null;
     checkSuperuserPrivilege();
@@ -8760,7 +8763,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       checkNameNodeSafeMode("Cannot create encryption zone on " + src);
       src = resolvePath(src, pathComponents);
 
-      final XAttr ezXAttr = dir.createEncryptionZone(src, keyName);
+      final CipherSuite suite = CipherSuite.convert(cipher);
+      final XAttr ezXAttr = dir.createEncryptionZone(src, suite, keyName);
       List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
       xAttrs.add(ezXAttr);
       getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java

@@ -133,6 +133,7 @@ public class INodesInPath {
    *        be thrown when the path refers to a symbolic link.
    * @return the specified number of existing INodes in the path
    */
+  // TODO: Eliminate null elements from inodes (to be provided by HDFS-7104)
   static INodesInPath resolve(final INodeDirectory startingDir,
       final byte[][] components, final int numOfINodes, 
       final boolean resolveLink) throws UnresolvedLinkException {
@@ -311,7 +312,7 @@ public class INodesInPath {
   }
 
   /**
-   * @return the whole inodes array including the null elements.
+   * @return the inodes array excluding the null elements.
    */
   INode[] getINodes() {
     if (capacity < inodes.length) {

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java

@@ -38,12 +38,14 @@ public abstract class AbstractNNFailoverProxyProvider<T> implements
   public abstract boolean useLogicalURI(); 
 
   /**
-   * Set for tracking if a secure client falls back to simple auth.
+   * Set for tracking if a secure client falls back to simple auth.  This method
+   * is synchronized only to stifle a Findbugs warning.
    *
    * @param fallbackToSimpleAuth - set to true or false during this method to
    *   indicate if a secure client falls back to simple auth
    */
-  public void setFallbackToSimpleAuth(AtomicBoolean fallbackToSimpleAuth) {
+  public synchronized void setFallbackToSimpleAuth(
+      AtomicBoolean fallbackToSimpleAuth) {
     this.fallbackToSimpleAuth = fallbackToSimpleAuth;
   }
 }

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/encryption.proto

@@ -46,9 +46,10 @@ message ListEncryptionZonesRequestProto {
 }
 
 message EncryptionZoneProto {
-  required string path = 1;
-  required string keyName = 2;
-  required int64 id = 3;
+  required int64 id = 1;
+  required string path = 2;
+  required CipherSuite suite = 3;
+  required string keyName = 4;
 }
 
 message ListEncryptionZonesResponseProto {

+ 21 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto

@@ -217,7 +217,27 @@ message FileEncryptionInfoProto {
   required CipherSuite suite = 1;
   required bytes key = 2;
   required bytes iv = 3;
-  required string ezKeyVersionName = 4;
+  required string keyName = 4;
+  required string ezKeyVersionName = 5;
+}
+
+/**
+ * Encryption information for an individual
+ * file within an encryption zone
+ */
+message PerFileEncryptionInfoProto {
+  required bytes key = 1;
+  required bytes iv = 2;
+  required string ezKeyVersionName = 3;
+}
+
+/**
+ * Encryption information for an encryption
+ * zone
+ */
+message ZoneEncryptionInfoProto {
+  required CipherSuite suite = 1;
+  required string keyName = 2;
 }
 
 /**

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java

@@ -116,8 +116,8 @@ public class TestEncryptionZones {
   protected FileContextTestWrapper fcWrapper;
 
   protected String getKeyProviderURI() {
-    return JavaKeyStoreProvider.SCHEME_NAME + "://file" + testRootDir +
-        "/test.jks";
+    return JavaKeyStoreProvider.SCHEME_NAME + "://file" +
+      new Path(testRootDir.toString(), "test.jks").toUri();
   }
 
   @Before
@@ -1043,7 +1043,7 @@ public class TestEncryptionZones {
     dfsAdmin.createEncryptionZone(zone, TEST_KEY);
     DFSTestUtil.createFile(fs, zoneFile, len, (short) 1, 0xFEED);
     String contents = DFSTestUtil.readFile(fs, zoneFile);
-    final Path snap1 = fs.createSnapshot(zoneParent);
+    final Path snap1 = fs.createSnapshot(zoneParent, "snap1");
     final Path snap1Zone = new Path(snap1, zone.getName());
     assertEquals("Got unexpected ez path", zone.toString(),
         dfsAdmin.getEncryptionZoneForPath(snap1Zone).getPath().toString());
@@ -1052,14 +1052,14 @@ public class TestEncryptionZones {
     // snapshot
     fsWrapper.delete(zone, true);
     fsWrapper.mkdir(zone, FsPermission.getDirDefault(), true);
-    final Path snap2 = fs.createSnapshot(zoneParent);
+    final Path snap2 = fs.createSnapshot(zoneParent, "snap2");
     final Path snap2Zone = new Path(snap2, zone.getName());
     assertNull("Expected null ez path",
         dfsAdmin.getEncryptionZoneForPath(snap2Zone));
 
     // Create the encryption zone again
     dfsAdmin.createEncryptionZone(zone, TEST_KEY2);
-    final Path snap3 = fs.createSnapshot(zoneParent);
+    final Path snap3 = fs.createSnapshot(zoneParent, "snap3");
     final Path snap3Zone = new Path(snap3, zone.getName());
     // Check that snap3's EZ has the correct settings
     EncryptionZone ezSnap3 = dfsAdmin.getEncryptionZoneForPath(snap3Zone);

+ 61 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java

@@ -31,13 +31,19 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Test;
 
 import java.io.File;
 import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileLock;
+import java.nio.channels.OverlappingFileLockException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -46,13 +52,19 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 public class TestDataNodeHotSwapVolumes {
+  private static final Log LOG = LogFactory.getLog(
+    TestDataNodeHotSwapVolumes.class);
   private static final int BLOCK_SIZE = 512;
   private MiniDFSCluster cluster;
 
@@ -179,8 +191,10 @@ public class TestDataNodeHotSwapVolumes {
     DataNode.ChangedVolumes changedVolumes =dn.parseChangedVolumes();
     List<StorageLocation> newVolumes = changedVolumes.newLocations;
     assertEquals(2, newVolumes.size());
-    assertEquals("/foo/path1", newVolumes.get(0).getFile().getAbsolutePath());
-    assertEquals("/foo/path2", newVolumes.get(1).getFile().getAbsolutePath());
+    assertEquals(new File("/foo/path1").getAbsolutePath(),
+      newVolumes.get(0).getFile().getAbsolutePath());
+    assertEquals(new File("/foo/path2").getAbsolutePath(),
+      newVolumes.get(1).getFile().getAbsolutePath());
 
     List<StorageLocation> removedVolumes = changedVolumes.deactivateLocations;
     assertEquals(oldLocations.size(), removedVolumes.size());
@@ -371,6 +385,8 @@ public class TestDataNodeHotSwapVolumes {
     String newDirs = oldDirs.iterator().next();  // Keep the first volume.
     dn.reconfigurePropertyImpl(
         DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, newDirs);
+    assertFileLocksReleased(
+      new ArrayList<String>(oldDirs).subList(1, oldDirs.size()));
     dn.scheduleAllBlockReport(0);
 
     try {
@@ -409,6 +425,8 @@ public class TestDataNodeHotSwapVolumes {
     String newDirs = oldDirs.iterator().next();  // Keep the first volume.
     dn.reconfigurePropertyImpl(
         DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, newDirs);
+    assertFileLocksReleased(
+      new ArrayList<String>(oldDirs).subList(1, oldDirs.size()));
 
     // Force DataNode to report missing blocks.
     dn.scheduleAllBlockReport(0);
@@ -420,4 +438,44 @@ public class TestDataNodeHotSwapVolumes {
     // Wait NameNode to replica missing blocks.
     DFSTestUtil.waitReplication(fs, testFile, replFactor);
   }
-}
+
+  /**
+   * Asserts that the storage lock file in each given directory has been
+   * released.  This method works by trying to acquire the lock file itself.  If
+   * locking fails here, then the main code must have failed to release it.
+   *
+   * @param dirs every storage directory to check
+   * @throws IOException if there is an unexpected I/O error
+   */
+  private static void assertFileLocksReleased(Collection<String> dirs)
+      throws IOException {
+    for (String dir: dirs) {
+      StorageLocation sl = StorageLocation.parse(dir);
+      File lockFile = new File(sl.getFile(), Storage.STORAGE_FILE_LOCK);
+      RandomAccessFile raf = null;
+      FileChannel channel = null;
+      FileLock lock = null;
+      try {
+        raf = new RandomAccessFile(lockFile, "rws");
+        channel = raf.getChannel();
+        lock = channel.tryLock();
+        assertNotNull(String.format(
+          "Lock file at %s appears to be held by a different process.",
+          lockFile.getAbsolutePath()), lock);
+      } catch (OverlappingFileLockException e) {
+        fail(String.format("Must release lock file at %s.",
+          lockFile.getAbsolutePath()));
+      } finally {
+        if (lock != null) {
+          try {
+            lock.release();
+          } catch (IOException e) {
+            LOG.warn(String.format("I/O error releasing file lock %s.",
+              lockFile.getAbsolutePath()), e);
+          }
+        }
+        IOUtils.cleanup(null, channel, raf);
+      }
+    }
+  }
+}

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataStorage.java

@@ -63,6 +63,7 @@ public class TestDataStorage {
 
   @After
   public void tearDown() throws IOException {
+    storage.unlockAll();
     FileUtil.fullyDelete(TEST_DIR);
   }
 

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCacheRevocation.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
+import static org.junit.Assume.assumeTrue;
+
 import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.EnumSet;
@@ -41,6 +43,7 @@ import org.apache.hadoop.io.nativeio.NativeIO.POSIX.CacheManipulator;
 import org.apache.hadoop.io.nativeio.NativeIO.POSIX.NoMlockCacheManipulator;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
+import org.apache.hadoop.util.NativeCodeLoader;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -94,6 +97,7 @@ public class TestFsDatasetCacheRevocation {
    */
   @Test(timeout=120000)
   public void testPinning() throws Exception {
+    assumeTrue(NativeCodeLoader.isNativeCodeLoaded() && !Path.WINDOWS);
     Configuration conf = getDefaultConf();
     // Set a really long revocation timeout, so that we won't reach it during
     // this test.
@@ -143,6 +147,7 @@ public class TestFsDatasetCacheRevocation {
    */
   @Test(timeout=120000)
   public void testRevocation() throws Exception {
+    assumeTrue(NativeCodeLoader.isNativeCodeLoaded() && !Path.WINDOWS);
     BlockReaderTestUtil.enableHdfsCachingTracing();
     BlockReaderTestUtil.enableShortCircuitShmTracing();
     Configuration conf = getDefaultConf();

+ 9 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java

@@ -53,13 +53,15 @@ public class TestTracing {
   private static Configuration conf;
   private static MiniDFSCluster cluster;
   private static DistributedFileSystem dfs;
+  private static SpanReceiverHost spanReceiverHost;
 
   @Test
-  public void testSpanReceiverHost() throws Exception {
-    Configuration conf = new Configuration();
-    conf.set(SpanReceiverHost.SPAN_RECEIVERS_CONF_KEY,
-        SetSpanReceiver.class.getName());
-    SpanReceiverHost spanReceiverHost = SpanReceiverHost.getInstance(conf);
+  public void testGetSpanReceiverHost() throws Exception {
+    Configuration c = new Configuration();
+    // getting instance already loaded.
+    c.set(SpanReceiverHost.SPAN_RECEIVERS_CONF_KEY, "");
+    SpanReceiverHost s = SpanReceiverHost.getInstance(c);
+    Assert.assertEquals(spanReceiverHost, s);
   }
 
   @Test
@@ -228,8 +230,10 @@ public class TestTracing {
     cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(3)
         .build();
+    cluster.waitActive();
 
     dfs = cluster.getFileSystem();
+    spanReceiverHost = SpanReceiverHost.getInstance(conf);
   }
 
   @AfterClass

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.tracing;
 
+import static org.junit.Assume.assumeTrue;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
@@ -27,6 +29,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
+import org.apache.hadoop.util.NativeCodeLoader;
 import org.htrace.Sampler;
 import org.htrace.Span;
 import org.htrace.Trace;
@@ -59,6 +62,7 @@ public class TestTracingShortCircuitLocalRead {
 
   @Test
   public void testShortCircuitTraceHooks() throws IOException {
+    assumeTrue(NativeCodeLoader.isNativeCodeLoaded() && !Path.WINDOWS);
     conf = new Configuration();
     conf.set(SpanReceiverHost.SPAN_RECEIVERS_CONF_KEY,
         TestTracing.SetSpanReceiver.class.getName());

+ 7 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -254,6 +254,10 @@ Release 2.6.0 - UNRELEASED
     MAPREDUCE-5891. Improved shuffle error handling across NM restarts
     (Junping Du via jlowe)
 
+    MAPREDUCE-5279. Made MR headroom calculation honor cpu dimension when YARN
+    scheduler resource type is memory plus cpu. (Peng Zhang and Varun Vasudev
+    via zjshen)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -360,6 +364,9 @@ Release 2.6.0 - UNRELEASED
     ApplicationNotFoundException if the job rolled off the RM view (Sangjin
     Lee via jlowe)
 
+    MAPREDUCE-6095. Enable DistributedCache for uber-mode Jobs (Gera Shegalov
+    via jlowe)
+
 Release 2.5.1 - 2014-09-05
 
   INCOMPATIBLE CHANGES

+ 1 - 62
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java

@@ -23,15 +23,11 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.net.InetSocketAddress;
-import java.net.URI;
 import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.concurrent.ScheduledExecutorService;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSError;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalDirAllocator;
@@ -43,7 +39,6 @@ import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.counters.Limits;
-import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
 import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
@@ -307,7 +302,7 @@ class YarnChild {
     task.localizeConfiguration(job);
 
     // Set up the DistributedCache related configs
-    setupDistributedCacheConfig(job);
+    MRApps.setupDistributedCacheLocal(job);
 
     // Overwrite the localized task jobconf which is linked to in the current
     // work-dir.
@@ -317,62 +312,6 @@ class YarnChild {
     task.setConf(job);
   }
 
-  /**
-   * Set up the DistributedCache related configs to make
-   * {@link DistributedCache#getLocalCacheFiles(Configuration)}
-   * and
-   * {@link DistributedCache#getLocalCacheArchives(Configuration)}
-   * working.
-   * @param job
-   * @throws IOException
-   */
-  private static void setupDistributedCacheConfig(final JobConf job)
-      throws IOException {
-
-    String localWorkDir = System.getenv("PWD");
-    //        ^ ^ all symlinks are created in the current work-dir
-
-    // Update the configuration object with localized archives.
-    URI[] cacheArchives = DistributedCache.getCacheArchives(job);
-    if (cacheArchives != null) {
-      List<String> localArchives = new ArrayList<String>();
-      for (int i = 0; i < cacheArchives.length; ++i) {
-        URI u = cacheArchives[i];
-        Path p = new Path(u);
-        Path name =
-            new Path((null == u.getFragment()) ? p.getName()
-                : u.getFragment());
-        String linkName = name.toUri().getPath();
-        localArchives.add(new Path(localWorkDir, linkName).toUri().getPath());
-      }
-      if (!localArchives.isEmpty()) {
-        job.set(MRJobConfig.CACHE_LOCALARCHIVES, StringUtils
-            .arrayToString(localArchives.toArray(new String[localArchives
-                .size()])));
-      }
-    }
-
-    // Update the configuration object with localized files.
-    URI[] cacheFiles = DistributedCache.getCacheFiles(job);
-    if (cacheFiles != null) {
-      List<String> localFiles = new ArrayList<String>();
-      for (int i = 0; i < cacheFiles.length; ++i) {
-        URI u = cacheFiles[i];
-        Path p = new Path(u);
-        Path name =
-            new Path((null == u.getFragment()) ? p.getName()
-                : u.getFragment());
-        String linkName = name.toUri().getPath();
-        localFiles.add(new Path(localWorkDir, linkName).toUri().getPath());
-      }
-      if (!localFiles.isEmpty()) {
-        job.set(MRJobConfig.CACHE_LOCALFILES,
-            StringUtils.arrayToString(localFiles
-                .toArray(new String[localFiles.size()])));
-      }
-    }
-  }
-
   private static final FsPermission urw_gr =
     FsPermission.createImmutable((short) 0640);
 

+ 1 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

@@ -826,6 +826,7 @@ public class MRAppMaster extends CompositeService {
     @Override
     protected void serviceStart() throws Exception {
       if (job.isUber()) {
+        MRApps.setupDistributedCacheLocal(getConfig());
         this.containerAllocator = new LocalContainerAllocator(
             this.clientService, this.context, nmHost, nmPort, nmHttpPort
             , containerID);

+ 11 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.mapreduce.v2.app.rm;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
+import java.util.EnumSet;
 import java.util.Map;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -59,6 +60,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
 
 /**
  * Registers/unregisters to RM and sends heartbeats to RM.
@@ -90,6 +92,8 @@ public abstract class RMCommunicator extends AbstractService
   private volatile boolean shouldUnregister = true;
   private boolean isApplicationMasterRegistered = false;
 
+  private EnumSet<SchedulerResourceTypes> schedulerResourceTypes;
+
   public RMCommunicator(ClientService clientService, AppContext context) {
     super("RMCommunicator");
     this.clientService = clientService;
@@ -98,6 +102,7 @@ public abstract class RMCommunicator extends AbstractService
     this.applicationId = context.getApplicationID();
     this.stopped = new AtomicBoolean(false);
     this.heartbeatCallbacks = new ConcurrentLinkedQueue<Runnable>();
+    this.schedulerResourceTypes = EnumSet.of(SchedulerResourceTypes.MEMORY);
   }
 
   @Override
@@ -163,10 +168,11 @@ public abstract class RMCommunicator extends AbstractService
         setClientToAMToken(response.getClientToAMTokenMasterKey());        
       }
       this.applicationACLs = response.getApplicationACLs();
-      LOG.info("maxContainerCapability: " + maxContainerCapability.getMemory());
+      LOG.info("maxContainerCapability: " + maxContainerCapability);
       String queue = response.getQueue();
       LOG.info("queue: " + queue);
       job.setQueueName(queue);
+      this.schedulerResourceTypes.addAll(response.getSchedulerResourceTypes());
     } catch (Exception are) {
       LOG.error("Exception while registering", are);
       throw new YarnRuntimeException(are);
@@ -343,4 +349,8 @@ public abstract class RMCommunicator extends AbstractService
   protected boolean isApplicationMasterRegistered() {
     return isApplicationMasterRegistered;
   }
+
+  public EnumSet<SchedulerResourceTypes> getSchedulerResourceTypes() {
+    return schedulerResourceTypes;
+  }
 }

+ 165 - 107
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java

@@ -73,6 +73,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.PreemptionMessage;
 import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.client.api.NMTokenCache;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -80,6 +81,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.RackResolver;
+import org.apache.hadoop.yarn.util.resource.Resources;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -149,8 +151,8 @@ public class RMContainerAllocator extends RMContainerRequestor
   private int lastCompletedTasks = 0;
   
   private boolean recalculateReduceSchedule = false;
-  private int mapResourceRequest;//memory
-  private int reduceResourceRequest;//memory
+  private Resource mapResourceRequest = Resources.none();
+  private Resource reduceResourceRequest = Resources.none();
   
   private boolean reduceStarted = false;
   private float maxReduceRampupLimit = 0;
@@ -328,49 +330,61 @@ public class RMContainerAllocator extends RMContainerRequestor
     if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) {
       ContainerRequestEvent reqEvent = (ContainerRequestEvent) event;
       JobId jobId = getJob().getID();
-      int supportedMaxContainerCapability =
-          getMaxContainerCapability().getMemory();
+      Resource supportedMaxContainerCapability = getMaxContainerCapability();
       if (reqEvent.getAttemptID().getTaskId().getTaskType().equals(TaskType.MAP)) {
-        if (mapResourceRequest == 0) {
-          mapResourceRequest = reqEvent.getCapability().getMemory();
-          eventHandler.handle(new JobHistoryEvent(jobId, 
-              new NormalizedResourceEvent(org.apache.hadoop.mapreduce.TaskType.MAP,
-                  mapResourceRequest)));
-          LOG.info("mapResourceRequest:"+ mapResourceRequest);
-          if (mapResourceRequest > supportedMaxContainerCapability) {
-            String diagMsg = "MAP capability required is more than the supported " +
-            "max container capability in the cluster. Killing the Job. mapResourceRequest: " +
-                mapResourceRequest + " maxContainerCapability:" + supportedMaxContainerCapability;
+        if (mapResourceRequest.equals(Resources.none())) {
+          mapResourceRequest = reqEvent.getCapability();
+          eventHandler.handle(new JobHistoryEvent(jobId,
+            new NormalizedResourceEvent(
+              org.apache.hadoop.mapreduce.TaskType.MAP, mapResourceRequest
+                .getMemory())));
+          LOG.info("mapResourceRequest:" + mapResourceRequest);
+          if (mapResourceRequest.getMemory() > supportedMaxContainerCapability
+            .getMemory()
+              || mapResourceRequest.getVirtualCores() > supportedMaxContainerCapability
+                .getVirtualCores()) {
+            String diagMsg =
+                "MAP capability required is more than the supported "
+                    + "max container capability in the cluster. Killing the Job. mapResourceRequest: "
+                    + mapResourceRequest + " maxContainerCapability:"
+                    + supportedMaxContainerCapability;
             LOG.info(diagMsg);
-            eventHandler.handle(new JobDiagnosticsUpdateEvent(
-                jobId, diagMsg));
+            eventHandler.handle(new JobDiagnosticsUpdateEvent(jobId, diagMsg));
             eventHandler.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
           }
         }
-        //set the rounded off memory
-        reqEvent.getCapability().setMemory(mapResourceRequest);
+        // set the resources
+        reqEvent.getCapability().setMemory(mapResourceRequest.getMemory());
+        reqEvent.getCapability().setVirtualCores(
+          mapResourceRequest.getVirtualCores());
         scheduledRequests.addMap(reqEvent);//maps are immediately scheduled
       } else {
-        if (reduceResourceRequest == 0) {
-          reduceResourceRequest = reqEvent.getCapability().getMemory();
-          eventHandler.handle(new JobHistoryEvent(jobId, 
-              new NormalizedResourceEvent(
-                  org.apache.hadoop.mapreduce.TaskType.REDUCE,
-                  reduceResourceRequest)));
-          LOG.info("reduceResourceRequest:"+ reduceResourceRequest);
-          if (reduceResourceRequest > supportedMaxContainerCapability) {
-            String diagMsg = "REDUCE capability required is more than the " +
-            		"supported max container capability in the cluster. Killing the " +
-            		"Job. reduceResourceRequest: " + reduceResourceRequest +
-            		" maxContainerCapability:" + supportedMaxContainerCapability;
+        if (reduceResourceRequest.equals(Resources.none())) {
+          reduceResourceRequest = reqEvent.getCapability();
+          eventHandler.handle(new JobHistoryEvent(jobId,
+            new NormalizedResourceEvent(
+              org.apache.hadoop.mapreduce.TaskType.REDUCE,
+              reduceResourceRequest.getMemory())));
+          LOG.info("reduceResourceRequest:" + reduceResourceRequest);
+          if (reduceResourceRequest.getMemory() > supportedMaxContainerCapability
+            .getMemory()
+              || reduceResourceRequest.getVirtualCores() > supportedMaxContainerCapability
+                .getVirtualCores()) {
+            String diagMsg =
+                "REDUCE capability required is more than the "
+                    + "supported max container capability in the cluster. Killing the "
+                    + "Job. reduceResourceRequest: " + reduceResourceRequest
+                    + " maxContainerCapability:"
+                    + supportedMaxContainerCapability;
             LOG.info(diagMsg);
-            eventHandler.handle(new JobDiagnosticsUpdateEvent(
-                jobId, diagMsg));
+            eventHandler.handle(new JobDiagnosticsUpdateEvent(jobId, diagMsg));
             eventHandler.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
           }
         }
-        //set the rounded off memory
-        reqEvent.getCapability().setMemory(reduceResourceRequest);
+        // set the resources
+        reqEvent.getCapability().setMemory(reduceResourceRequest.getMemory());
+        reqEvent.getCapability().setVirtualCores(
+          reduceResourceRequest.getVirtualCores());
         if (reqEvent.getEarlierAttemptFailed()) {
           //add to the front of queue for fail fast
           pendingReduces.addFirst(new ContainerRequest(reqEvent, PRIORITY_REDUCE));
@@ -425,34 +439,40 @@ public class RMContainerAllocator extends RMContainerRequestor
 
   @Private
   @VisibleForTesting
-  synchronized void setReduceResourceRequest(int mem) {
-    this.reduceResourceRequest = mem;
+  synchronized void setReduceResourceRequest(Resource res) {
+    this.reduceResourceRequest = res;
   }
 
   @Private
   @VisibleForTesting
-  synchronized void setMapResourceRequest(int mem) {
-    this.mapResourceRequest = mem;
+  synchronized void setMapResourceRequest(Resource res) {
+    this.mapResourceRequest = res;
   }
 
   @Private
   @VisibleForTesting
   void preemptReducesIfNeeded() {
-    if (reduceResourceRequest == 0) {
-      return; //no reduces
+    if (reduceResourceRequest.equals(Resources.none())) {
+      return; // no reduces
     }
     //check if reduces have taken over the whole cluster and there are 
     //unassigned maps
     if (scheduledRequests.maps.size() > 0) {
-      int memLimit = getMemLimit();
-      int availableMemForMap = memLimit - ((assignedRequests.reduces.size() -
-          assignedRequests.preemptionWaitingReduces.size()) * reduceResourceRequest);
-      //availableMemForMap must be sufficient to run atleast 1 map
-      if (availableMemForMap < mapResourceRequest) {
-        //to make sure new containers are given to maps and not reduces
-        //ramp down all scheduled reduces if any
-        //(since reduces are scheduled at higher priority than maps)
-        LOG.info("Ramping down all scheduled reduces:" + scheduledRequests.reduces.size());
+      Resource resourceLimit = getResourceLimit();
+      Resource availableResourceForMap =
+          Resources.subtract(
+            resourceLimit,
+            Resources.multiply(reduceResourceRequest,
+              assignedRequests.reduces.size()
+                  - assignedRequests.preemptionWaitingReduces.size()));
+      // availableMemForMap must be sufficient to run at least 1 map
+      if (ResourceCalculatorUtils.computeAvailableContainers(availableResourceForMap,
+        mapResourceRequest, getSchedulerResourceTypes()) <= 0) {
+        // to make sure new containers are given to maps and not reduces
+        // ramp down all scheduled reduces if any
+        // (since reduces are scheduled at higher priority than maps)
+        LOG.info("Ramping down all scheduled reduces:"
+            + scheduledRequests.reduces.size());
         for (ContainerRequest req : scheduledRequests.reduces.values()) {
           pendingReduces.add(req);
         }
@@ -462,17 +482,25 @@ public class RMContainerAllocator extends RMContainerRequestor
         //hanging around for a while
         int hangingMapRequests = getNumOfHangingRequests(scheduledRequests.maps);
         if (hangingMapRequests > 0) {
-          //preempt for making space for at least one map
-          int premeptionLimit = Math.max(mapResourceRequest,
-              (int) (maxReducePreemptionLimit * memLimit));
-
-          int preemptMem = Math.min(hangingMapRequests * mapResourceRequest,
-              premeptionLimit);
-
-          int toPreempt = (int) Math.ceil((float) preemptMem / reduceResourceRequest);
-          toPreempt = Math.min(toPreempt, assignedRequests.reduces.size());
-
-          LOG.info("Going to preempt " + toPreempt + " due to lack of space for maps");
+          // preempt for making space for at least one map
+          int preemptionReduceNumForOneMap =
+              ResourceCalculatorUtils.divideAndCeilContainers(mapResourceRequest,
+                reduceResourceRequest, getSchedulerResourceTypes());
+          int preemptionReduceNumForPreemptionLimit =
+              ResourceCalculatorUtils.divideAndCeilContainers(
+                Resources.multiply(resourceLimit, maxReducePreemptionLimit),
+                reduceResourceRequest, getSchedulerResourceTypes());
+          int preemptionReduceNumForAllMaps =
+              ResourceCalculatorUtils.divideAndCeilContainers(
+                Resources.multiply(mapResourceRequest, hangingMapRequests),
+                reduceResourceRequest, getSchedulerResourceTypes());
+          int toPreempt =
+              Math.min(Math.max(preemptionReduceNumForOneMap,
+                preemptionReduceNumForPreemptionLimit),
+                preemptionReduceNumForAllMaps);
+
+          LOG.info("Going to preempt " + toPreempt
+              + " due to lack of space for maps");
           assignedRequests.preemptReduce(toPreempt);
         }
       }
@@ -497,7 +525,7 @@ public class RMContainerAllocator extends RMContainerRequestor
       int totalMaps, int completedMaps,
       int scheduledMaps, int scheduledReduces,
       int assignedMaps, int assignedReduces,
-      int mapResourceReqt, int reduceResourceReqt,
+      Resource mapResourceReqt, Resource reduceResourceReqt,
       int numPendingReduces,
       float maxReduceRampupLimit, float reduceSlowStart) {
     
@@ -505,8 +533,12 @@ public class RMContainerAllocator extends RMContainerRequestor
       return;
     }
     
-    int headRoom = getAvailableResources() != null ?
-        getAvailableResources().getMemory() : 0;
+    // get available resources for this job
+    Resource headRoom = getAvailableResources();
+    if (headRoom == null) {
+      headRoom = Resources.none();
+    }
+
     LOG.info("Recalculating schedule, headroom=" + headRoom);
     
     //check for slow start
@@ -540,49 +572,60 @@ public class RMContainerAllocator extends RMContainerRequestor
       completedMapPercent = 1;
     }
     
-    int netScheduledMapMem = 
-        (scheduledMaps + assignedMaps) * mapResourceReqt;
+    Resource netScheduledMapResource =
+        Resources.multiply(mapResourceReqt, (scheduledMaps + assignedMaps));
 
-    int netScheduledReduceMem = 
-        (scheduledReduces + assignedReduces) * reduceResourceReqt;
+    Resource netScheduledReduceResource =
+        Resources.multiply(reduceResourceReqt,
+          (scheduledReduces + assignedReduces));
+
+    Resource finalMapResourceLimit;
+    Resource finalReduceResourceLimit;
 
-    int finalMapMemLimit = 0;
-    int finalReduceMemLimit = 0;
-    
     // ramp up the reduces based on completed map percentage
-    int totalMemLimit = getMemLimit();
-    int idealReduceMemLimit = 
-        Math.min(
-            (int)(completedMapPercent * totalMemLimit),
-            (int) (maxReduceRampupLimit * totalMemLimit));
-    int idealMapMemLimit = totalMemLimit - idealReduceMemLimit;
+    Resource totalResourceLimit = getResourceLimit();
+
+    Resource idealReduceResourceLimit =
+        Resources.multiply(totalResourceLimit,
+          Math.min(completedMapPercent, maxReduceRampupLimit));
+    Resource ideaMapResourceLimit =
+        Resources.subtract(totalResourceLimit, idealReduceResourceLimit);
 
     // check if there aren't enough maps scheduled, give the free map capacity
-    // to reduce
-    if (idealMapMemLimit > netScheduledMapMem) {
-      int unusedMapMemLimit = idealMapMemLimit - netScheduledMapMem;
-      finalReduceMemLimit = idealReduceMemLimit + unusedMapMemLimit;
-      finalMapMemLimit = totalMemLimit - finalReduceMemLimit;
+    // to reduce.
+    // Even when container number equals, there may be unused resources in one
+    // dimension
+    if (ResourceCalculatorUtils.computeAvailableContainers(ideaMapResourceLimit,
+      mapResourceReqt, getSchedulerResourceTypes()) >= (scheduledMaps + assignedMaps)) {
+      // enough resource given to maps, given the remaining to reduces
+      Resource unusedMapResourceLimit =
+          Resources.subtract(ideaMapResourceLimit, netScheduledMapResource);
+      finalReduceResourceLimit =
+          Resources.add(idealReduceResourceLimit, unusedMapResourceLimit);
+      finalMapResourceLimit =
+          Resources.subtract(totalResourceLimit, finalReduceResourceLimit);
     } else {
-      finalMapMemLimit = idealMapMemLimit;
-      finalReduceMemLimit = idealReduceMemLimit;
+      finalMapResourceLimit = ideaMapResourceLimit;
+      finalReduceResourceLimit = idealReduceResourceLimit;
     }
-    
-    LOG.info("completedMapPercent " + completedMapPercent +
-        " totalMemLimit:" + totalMemLimit +
-        " finalMapMemLimit:" + finalMapMemLimit +
-        " finalReduceMemLimit:" + finalReduceMemLimit + 
-        " netScheduledMapMem:" + netScheduledMapMem +
-        " netScheduledReduceMem:" + netScheduledReduceMem);
-    
-    int rampUp = 
-        (finalReduceMemLimit - netScheduledReduceMem) / reduceResourceReqt;
-    
+
+    LOG.info("completedMapPercent " + completedMapPercent
+        + " totalResourceLimit:" + totalResourceLimit
+        + " finalMapResourceLimit:" + finalMapResourceLimit
+        + " finalReduceResourceLimit:" + finalReduceResourceLimit
+        + " netScheduledMapResource:" + netScheduledMapResource
+        + " netScheduledReduceResource:" + netScheduledReduceResource);
+
+    int rampUp =
+        ResourceCalculatorUtils.computeAvailableContainers(Resources.subtract(
+                finalReduceResourceLimit, netScheduledReduceResource),
+            reduceResourceReqt, getSchedulerResourceTypes());
+
     if (rampUp > 0) {
       rampUp = Math.min(rampUp, numPendingReduces);
       LOG.info("Ramping up " + rampUp);
       rampUpReduces(rampUp);
-    } else if (rampUp < 0){
+    } else if (rampUp < 0) {
       int rampDown = -1 * rampUp;
       rampDown = Math.min(rampDown, scheduledReduces);
       LOG.info("Ramping down " + rampDown);
@@ -618,8 +661,10 @@ public class RMContainerAllocator extends RMContainerRequestor
   
   @SuppressWarnings("unchecked")
   private List<Container> getResources() throws Exception {
-    int headRoom = getAvailableResources() != null
-        ? getAvailableResources().getMemory() : 0;//first time it would be null
+    // will be null the first time
+    Resource headRoom =
+        getAvailableResources() == null ? Resources.none() :
+            Resources.clone(getAvailableResources());
     AllocateResponse response;
     /*
      * If contact with RM is lost, the AM will wait MR_AM_TO_RM_WAIT_INTERVAL_MS
@@ -670,7 +715,9 @@ public class RMContainerAllocator extends RMContainerRequestor
         throw new YarnRuntimeException(msg);
       }
     }
-    int newHeadRoom = getAvailableResources() != null ? getAvailableResources().getMemory() : 0;
+    Resource newHeadRoom =
+        getAvailableResources() == null ? Resources.none()
+            : getAvailableResources();
     List<Container> newContainers = response.getAllocatedContainers();
     // Setting NMTokens
     if (response.getNMTokens() != null) {
@@ -694,10 +741,11 @@ public class RMContainerAllocator extends RMContainerRequestor
           new PreemptionContext(assignedRequests), preemptReq);
     }
 
-    if (newContainers.size() + finishedContainers.size() > 0 || headRoom != newHeadRoom) {
+    if (newContainers.size() + finishedContainers.size() > 0
+        || !headRoom.equals(newHeadRoom)) {
       //something changed
       recalculateReduceSchedule = true;
-      if (LOG.isDebugEnabled() && headRoom != newHeadRoom) {
+      if (LOG.isDebugEnabled() && !headRoom.equals(newHeadRoom)) {
         LOG.debug("headroom=" + newHeadRoom);
       }
     }
@@ -802,10 +850,18 @@ public class RMContainerAllocator extends RMContainerRequestor
   }
 
   @Private
-  public int getMemLimit() {
-    int headRoom = getAvailableResources() != null ? getAvailableResources().getMemory() : 0;
-    return headRoom + assignedRequests.maps.size() * mapResourceRequest +
-       assignedRequests.reduces.size() * reduceResourceRequest;
+  public Resource getResourceLimit() {
+    Resource headRoom = getAvailableResources();
+    if (headRoom == null) {
+      headRoom = Resources.none();
+    }
+    Resource assignedMapResource =
+        Resources.multiply(mapResourceRequest, assignedRequests.maps.size());
+    Resource assignedReduceResource =
+        Resources.multiply(reduceResourceRequest,
+          assignedRequests.reduces.size());
+    return Resources.add(headRoom,
+      Resources.add(assignedMapResource, assignedReduceResource));
   }
 
   @Private
@@ -914,10 +970,11 @@ public class RMContainerAllocator extends RMContainerRequestor
         // a container to be assigned
         boolean isAssignable = true;
         Priority priority = allocated.getPriority();
-        int allocatedMemory = allocated.getResource().getMemory();
+        Resource allocatedResource = allocated.getResource();
         if (PRIORITY_FAST_FAIL_MAP.equals(priority) 
             || PRIORITY_MAP.equals(priority)) {
-          if (allocatedMemory < mapResourceRequest
+          if (ResourceCalculatorUtils.computeAvailableContainers(allocatedResource,
+              mapResourceRequest, getSchedulerResourceTypes()) <= 0
               || maps.isEmpty()) {
             LOG.info("Cannot assign container " + allocated 
                 + " for a map as either "
@@ -928,7 +985,8 @@ public class RMContainerAllocator extends RMContainerRequestor
           }
         } 
         else if (PRIORITY_REDUCE.equals(priority)) {
-          if (allocatedMemory < reduceResourceRequest
+          if (ResourceCalculatorUtils.computeAvailableContainers(allocatedResource,
+              reduceResourceRequest, getSchedulerResourceTypes()) <= 0
               || reduces.isEmpty()) {
             LOG.info("Cannot assign container " + allocated 
                 + " for a reduce as either "

+ 52 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/ResourceCalculatorUtils.java

@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce.v2.app.rm;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.EnumSet;
+
+public class ResourceCalculatorUtils {
+  public static int divideAndCeil(int a, int b) {
+    if (b == 0) {
+      return 0;
+    }
+    return (a + (b - 1)) / b;
+  }
+
+  public static int computeAvailableContainers(Resource available,
+      Resource required, EnumSet<SchedulerResourceTypes> resourceTypes) {
+    if (resourceTypes.contains(SchedulerResourceTypes.CPU)) {
+      return Math.min(available.getMemory() / required.getMemory(),
+        available.getVirtualCores() / required.getVirtualCores());
+    }
+    return available.getMemory() / required.getMemory();
+  }
+
+  public static int divideAndCeilContainers(Resource required, Resource factor,
+      EnumSet<SchedulerResourceTypes> resourceTypes) {
+    if (resourceTypes.contains(SchedulerResourceTypes.CPU)) {
+      return Math.max(divideAndCeil(required.getMemory(), factor.getMemory()),
+        divideAndCeil(required.getVirtualCores(), factor.getVirtualCores()));
+    }
+    return divideAndCeil(required.getMemory(), factor.getMemory());
+  }
+}

+ 54 - 28
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.mapreduce.v2.app.rm;
 
+import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyFloat;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Matchers.isA;
@@ -30,19 +31,14 @@ import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.ClusterInfo;
 import org.apache.hadoop.mapreduce.v2.app.MRApp;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.Assert;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -458,8 +454,8 @@ public class TestRMContainerAllocator {
             0, 0, 0, 0, 0, 0, "jobfile", null, false, ""));
     MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
         appAttemptId, mockJob, new SystemClock());
-    allocator.setMapResourceRequest(1024);
-    allocator.setReduceResourceRequest(1024);
+    allocator.setMapResourceRequest(BuilderUtils.newResource(1024, 1));
+    allocator.setReduceResourceRequest(BuilderUtils.newResource(1024, 1));
     RMContainerAllocator.AssignedRequests assignedRequests =
         allocator.getAssignedRequests();
     RMContainerAllocator.ScheduledRequests scheduledRequests =
@@ -478,7 +474,7 @@ public class TestRMContainerAllocator {
 
   @Test(timeout = 30000)
   public void testNonAggressivelyPreemptReducers() throws Exception {
-    LOG.info("Running testPreemptReducers");
+    LOG.info("Running testNonAggressivelyPreemptReducers");
 
     final int preemptThreshold = 2; //sec
     Configuration conf = new Configuration();
@@ -513,8 +509,8 @@ public class TestRMContainerAllocator {
     clock.setTime(1);
     MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
         appAttemptId, mockJob, clock);
-    allocator.setMapResourceRequest(1024);
-    allocator.setReduceResourceRequest(1024);
+    allocator.setMapResourceRequest(BuilderUtils.newResource(1024, 1));
+    allocator.setReduceResourceRequest(BuilderUtils.newResource(1024, 1));
     RMContainerAllocator.AssignedRequests assignedRequests =
         allocator.getAssignedRequests();
     RMContainerAllocator.ScheduledRequests scheduledRequests =
@@ -1774,17 +1770,19 @@ public class TestRMContainerAllocator {
     int scheduledReduces = 0;
     int assignedMaps = 2;
     int assignedReduces = 0;
-    int mapResourceReqt = 1024;
-    int reduceResourceReqt = 2*1024;
+    Resource mapResourceReqt = BuilderUtils.newResource(1024, 1);
+    Resource reduceResourceReqt = BuilderUtils.newResource(2 * 1024, 1);
     int numPendingReduces = 4;
     float maxReduceRampupLimit = 0.5f;
     float reduceSlowStart = 0.2f;
     
     RMContainerAllocator allocator = mock(RMContainerAllocator.class);
-    doCallRealMethod().when(allocator).
-        scheduleReduces(anyInt(), anyInt(), anyInt(), anyInt(), anyInt(), 
-            anyInt(), anyInt(), anyInt(), anyInt(), anyFloat(), anyFloat());
-    
+    doCallRealMethod().when(allocator).scheduleReduces(anyInt(), anyInt(),
+        anyInt(), anyInt(), anyInt(), anyInt(), any(Resource.class),
+        any(Resource.class), anyInt(), anyFloat(), anyFloat());
+    doReturn(EnumSet.of(SchedulerResourceTypes.MEMORY)).when(allocator)
+      .getSchedulerResourceTypes();
+
     // Test slow-start
     allocator.scheduleReduces(
         totalMaps, succeededMaps, 
@@ -1808,6 +1806,7 @@ public class TestRMContainerAllocator {
     verify(allocator, never()).scheduleAllReduces();
 
     succeededMaps = 3;
+    doReturn(BuilderUtils.newResource(0, 0)).when(allocator).getResourceLimit();
     allocator.scheduleReduces(
         totalMaps, succeededMaps, 
         scheduledMaps, scheduledReduces, 
@@ -1818,7 +1817,8 @@ public class TestRMContainerAllocator {
     verify(allocator, times(1)).setIsReduceStarted(true);
     
     // Test reduce ramp-up
-    doReturn(100 * 1024).when(allocator).getMemLimit();
+    doReturn(BuilderUtils.newResource(100 * 1024, 100 * 1)).when(allocator)
+      .getResourceLimit();
     allocator.scheduleReduces(
         totalMaps, succeededMaps, 
         scheduledMaps, scheduledReduces, 
@@ -1831,13 +1831,14 @@ public class TestRMContainerAllocator {
 
     // Test reduce ramp-down
     scheduledReduces = 3;
-    doReturn(10 * 1024).when(allocator).getMemLimit();
+    doReturn(BuilderUtils.newResource(10 * 1024, 10 * 1)).when(allocator)
+      .getResourceLimit();
     allocator.scheduleReduces(
-        totalMaps, succeededMaps, 
-        scheduledMaps, scheduledReduces, 
-        assignedMaps, assignedReduces, 
-        mapResourceReqt, reduceResourceReqt, 
-        numPendingReduces, 
+        totalMaps, succeededMaps,
+        scheduledMaps, scheduledReduces,
+        assignedMaps, assignedReduces,
+        mapResourceReqt, reduceResourceReqt,
+        numPendingReduces,
         maxReduceRampupLimit, reduceSlowStart);
     verify(allocator).rampDownReduces(anyInt());
 
@@ -1846,7 +1847,8 @@ public class TestRMContainerAllocator {
     // should be invoked twice.
     scheduledMaps = 2;
     assignedReduces = 2;
-    doReturn(10 * 1024).when(allocator).getMemLimit();
+    doReturn(BuilderUtils.newResource(10 * 1024, 10 * 1)).when(allocator)
+      .getResourceLimit();
     allocator.scheduleReduces(
         totalMaps, succeededMaps, 
         scheduledMaps, scheduledReduces, 
@@ -1855,6 +1857,30 @@ public class TestRMContainerAllocator {
         numPendingReduces, 
         maxReduceRampupLimit, reduceSlowStart);
     verify(allocator, times(2)).rampDownReduces(anyInt());
+
+    doReturn(
+        EnumSet.of(SchedulerResourceTypes.MEMORY, SchedulerResourceTypes.CPU))
+        .when(allocator).getSchedulerResourceTypes();
+
+    // Test ramp-down when enough memory but not enough cpu resource
+    scheduledMaps = 10;
+    assignedReduces = 0;
+    doReturn(BuilderUtils.newResource(100 * 1024, 5 * 1)).when(allocator)
+        .getResourceLimit();
+    allocator.scheduleReduces(totalMaps, succeededMaps, scheduledMaps,
+        scheduledReduces, assignedMaps, assignedReduces, mapResourceReqt,
+        reduceResourceReqt, numPendingReduces, maxReduceRampupLimit,
+        reduceSlowStart);
+    verify(allocator, times(3)).rampDownReduces(anyInt());
+
+    // Test ramp-down when enough cpu but not enough memory resource
+    doReturn(BuilderUtils.newResource(10 * 1024, 100 * 1)).when(allocator)
+        .getResourceLimit();
+    allocator.scheduleReduces(totalMaps, succeededMaps, scheduledMaps,
+        scheduledReduces, assignedMaps, assignedReduces, mapResourceReqt,
+        reduceResourceReqt, numPendingReduces, maxReduceRampupLimit,
+        reduceSlowStart);
+    verify(allocator, times(4)).rampDownReduces(anyInt());
   }
 
   private static class RecalculateContainerAllocator extends MyContainerAllocator {
@@ -1868,7 +1894,7 @@ public class TestRMContainerAllocator {
     @Override
     public void scheduleReduces(int totalMaps, int completedMaps,
         int scheduledMaps, int scheduledReduces, int assignedMaps,
-        int assignedReduces, int mapResourceReqt, int reduceResourceReqt,
+        int assignedReduces, Resource mapResourceReqt, Resource reduceResourceReqt,
         int numPendingReduces, float maxReduceRampupLimit, float reduceSlowStart) {
       recalculatedReduceSchedule = true;
     }

+ 57 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java

@@ -26,12 +26,11 @@ import java.net.URISyntaxException;
 import java.security.AccessController;
 import java.security.PrivilegedActionException;
 import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -58,8 +57,6 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.util.ApplicationClassLoader;
-import org.apache.hadoop.util.Shell;
-import org.apache.hadoop.util.StringInterner;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.ContainerLogAppender;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
@@ -469,6 +466,62 @@ public class MRApps extends Apps {
         DistributedCache.getFileVisibilities(conf));
   }
 
+  /**
+   * Set up the DistributedCache related configs to make
+   * {@link DistributedCache#getLocalCacheFiles(Configuration)}
+   * and
+   * {@link DistributedCache#getLocalCacheArchives(Configuration)}
+   * working.
+   * @param conf
+   * @throws java.io.IOException
+   */
+  public static void setupDistributedCacheLocal(Configuration conf)
+      throws IOException {
+
+    String localWorkDir = System.getenv("PWD");
+    //        ^ ^ all symlinks are created in the current work-dir
+
+    // Update the configuration object with localized archives.
+    URI[] cacheArchives = DistributedCache.getCacheArchives(conf);
+    if (cacheArchives != null) {
+      List<String> localArchives = new ArrayList<String>();
+      for (int i = 0; i < cacheArchives.length; ++i) {
+        URI u = cacheArchives[i];
+        Path p = new Path(u);
+        Path name =
+            new Path((null == u.getFragment()) ? p.getName()
+                : u.getFragment());
+        String linkName = name.toUri().getPath();
+        localArchives.add(new Path(localWorkDir, linkName).toUri().getPath());
+      }
+      if (!localArchives.isEmpty()) {
+        conf.set(MRJobConfig.CACHE_LOCALARCHIVES, StringUtils
+            .arrayToString(localArchives.toArray(new String[localArchives
+                .size()])));
+      }
+    }
+
+    // Update the configuration object with localized files.
+    URI[] cacheFiles = DistributedCache.getCacheFiles(conf);
+    if (cacheFiles != null) {
+      List<String> localFiles = new ArrayList<String>();
+      for (int i = 0; i < cacheFiles.length; ++i) {
+        URI u = cacheFiles[i];
+        Path p = new Path(u);
+        Path name =
+            new Path((null == u.getFragment()) ? p.getName()
+                : u.getFragment());
+        String linkName = name.toUri().getPath();
+        localFiles.add(new Path(localWorkDir, linkName).toUri().getPath());
+      }
+      if (!localFiles.isEmpty()) {
+        conf.set(MRJobConfig.CACHE_LOCALFILES,
+            StringUtils.arrayToString(localFiles
+                .toArray(new String[localFiles.size()])));
+      }
+    }
+  }
+
   private static String getResourceDescription(LocalResourceType type) {
     if(type == LocalResourceType.ARCHIVE || type == LocalResourceType.PATTERN) {
       return "cache archive (" + MRJobConfig.CACHE_ARCHIVES + ") ";

+ 0 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestUberAM.java

@@ -191,11 +191,4 @@ public class TestUberAM extends TestMRJobs {
   throws IOException, InterruptedException, ClassNotFoundException {
     super.testSleepJobWithSecurityOn();
   }
-
-  // Add a test for distcache when uber mode is enabled. TODO
-  @Override
-  @Test
-  public void testDistributedCache() throws Exception {
-    //
-  }
 }

+ 15 - 0
hadoop-yarn-project/CHANGES.txt

@@ -238,6 +238,12 @@ Release 2.6.0 - UNRELEASED
     YARN-2001. Added a time threshold for RM to wait before starting container
     allocations after restart/failover. (Jian He via vinodkv)
 
+    YARN-1372. Ensure all completed containers are reported to the AMs across
+    RM restart. (Anubhav Dhoot via jianhe)
+
+    YARN-2539. FairScheduler: Set the default value for maxAMShare to 0.5. 
+    (Wei Yan via kasha)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -406,6 +412,15 @@ Release 2.6.0 - UNRELEASED
     YARN-2460. Remove obsolete entries from yarn-default.xml (Ray Chiang via
     aw)
 
+    YARN-2452. TestRMApplicationHistoryWriter fails with FairScheduler. 
+    (Zhihai Xu via kasha)
+
+    YARN-2453. TestProportionalCapacityPreemptionPolicy fails with 
+    FairScheduler. (Zhihai Xu via kasha)
+    
+    YARN-2540. FairScheduler: Queue filters not working on scheduler page in 
+    RM UI. (Ashwin Shankar via kasha)
+
 Release 2.5.1 - 2014-09-05
 
   INCOMPATIBLE CHANGES

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java

@@ -18,12 +18,12 @@
 
 package org.apache.hadoop.yarn.util.resource;
 
-import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.util.Records;
 
-@Private
+@InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
 @Unstable
 public class Resources {
   

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java

@@ -30,6 +30,7 @@ public interface NodeHeartbeatResponse {
   NodeAction getNodeAction();
 
   List<ContainerId> getContainersToCleanup();
+  List<ContainerId> getFinishedContainersPulledByAM();
 
   List<ApplicationId> getApplicationsToCleanup();
 
@@ -43,6 +44,10 @@ public interface NodeHeartbeatResponse {
   void setNMTokenMasterKey(MasterKey secretKey);
 
   void addAllContainersToCleanup(List<ContainerId> containers);
+
+  // This tells NM to remove finished containers only after the AM
+  // has actually received it in a previous allocate response
+  void addFinishedContainersPulledByAM(List<ContainerId> containers);
   
   void addAllApplicationsToCleanup(List<ApplicationId> applications);
 

+ 67 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java

@@ -46,6 +46,7 @@ public class NodeHeartbeatResponsePBImpl extends ProtoBase<NodeHeartbeatResponse
   boolean viaProto = false;
   
   private List<ContainerId> containersToCleanup = null;
+  private List<ContainerId> finishedContainersPulledByAM = null;
   private List<ApplicationId> applicationsToCleanup = null;
   private MasterKey containerTokenMasterKey = null;
   private MasterKey nmTokenMasterKey = null;
@@ -73,6 +74,9 @@ public class NodeHeartbeatResponsePBImpl extends ProtoBase<NodeHeartbeatResponse
     if (this.applicationsToCleanup != null) {
       addApplicationsToCleanupToProto();
     }
+    if (this.finishedContainersPulledByAM != null) {
+      addFinishedContainersPulledByAMToProto();
+    }
     if (this.containerTokenMasterKey != null) {
       builder.setContainerTokenMasterKey(
           convertToProtoFormat(this.containerTokenMasterKey));
@@ -199,6 +203,12 @@ public class NodeHeartbeatResponsePBImpl extends ProtoBase<NodeHeartbeatResponse
     return this.containersToCleanup;
   }
 
+  @Override
+  public List<ContainerId> getFinishedContainersPulledByAM() {
+    initFinishedContainersPulledByAM();
+    return this.finishedContainersPulledByAM;
+  }
+
   private void initContainersToCleanup() {
     if (this.containersToCleanup != null) {
       return;
@@ -212,6 +222,19 @@ public class NodeHeartbeatResponsePBImpl extends ProtoBase<NodeHeartbeatResponse
     }
   }
 
+  private void initFinishedContainersPulledByAM() {
+    if (this.finishedContainersPulledByAM != null) {
+      return;
+    }
+    NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerIdProto> list = p.getFinishedContainersPulledByAmList();
+    this.finishedContainersPulledByAM = new ArrayList<ContainerId>();
+
+    for (ContainerIdProto c : list) {
+      this.finishedContainersPulledByAM.add(convertFromProtoFormat(c));
+    }
+  }
+
   @Override
   public void addAllContainersToCleanup(
       final List<ContainerId> containersToCleanup) {
@@ -221,6 +244,15 @@ public class NodeHeartbeatResponsePBImpl extends ProtoBase<NodeHeartbeatResponse
     this.containersToCleanup.addAll(containersToCleanup);
   }
 
+  @Override
+  public void addFinishedContainersPulledByAM(
+      final List<ContainerId> finishedContainersPulledByAM) {
+    if (finishedContainersPulledByAM == null)
+      return;
+    initFinishedContainersPulledByAM();
+    this.finishedContainersPulledByAM.addAll(finishedContainersPulledByAM);
+  }
+
   private void addContainersToCleanupToProto() {
     maybeInitBuilder();
     builder.clearContainersToCleanup();
@@ -256,6 +288,41 @@ public class NodeHeartbeatResponsePBImpl extends ProtoBase<NodeHeartbeatResponse
     builder.addAllContainersToCleanup(iterable);
   }
 
+  private void addFinishedContainersPulledByAMToProto() {
+    maybeInitBuilder();
+    builder.clearFinishedContainersPulledByAm();
+    if (finishedContainersPulledByAM == null)
+      return;
+    Iterable<ContainerIdProto> iterable = new Iterable<ContainerIdProto>() {
+
+      @Override
+      public Iterator<ContainerIdProto> iterator() {
+        return new Iterator<ContainerIdProto>() {
+
+          Iterator<ContainerId> iter = finishedContainersPulledByAM.iterator();
+
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+
+          @Override
+          public ContainerIdProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+
+          }
+        };
+
+      }
+    };
+    builder.addAllFinishedContainersPulledByAm(iterable);
+  }
+
   @Override
   public List<ApplicationId> getApplicationsToCleanup() {
     initApplicationsToCleanup();

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto

@@ -58,6 +58,7 @@ message NodeHeartbeatResponseProto {
   repeated ApplicationIdProto applications_to_cleanup = 6;
   optional int64 nextHeartBeatInterval = 7;
   optional string diagnostics_message = 8;
+  repeated ContainerIdProto finished_containers_pulled_by_am = 9;
 }
 
 message NMContainerStatusProto {

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java

@@ -311,7 +311,7 @@ public class NodeManager extends CompositeService
   public static class NMContext implements Context {
 
     private NodeId nodeId = null;
-    private final ConcurrentMap<ApplicationId, Application> applications =
+    protected final ConcurrentMap<ApplicationId, Application> applications =
         new ConcurrentHashMap<ApplicationId, Application>();
     protected final ConcurrentMap<ContainerId, Container> containers =
         new ConcurrentSkipListMap<ContainerId, Container>();

+ 48 - 32
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java

@@ -104,11 +104,6 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
   // Duration for which to track recently stopped container.
   private long durationToTrackStoppedContainers;
 
-  // This is used to track the current completed containers when nodeheartBeat
-  // is called. These completed containers will be removed from NM context after
-  // nodeHeartBeat succeeds and the response from the nodeHeartBeat is
-  // processed.
-  private final Set<ContainerId> previousCompletedContainers;
   private final NodeHealthCheckerService healthChecker;
   private final NodeManagerMetrics metrics;
 
@@ -125,7 +120,6 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     this.metrics = metrics;
     this.recentlyStoppedContainers =
         new LinkedHashMap<ContainerId, Long>();
-    this.previousCompletedContainers = new HashSet<ContainerId>();
   }
 
   @Override
@@ -331,7 +325,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     return appList;
   }
 
-  private NodeStatus getNodeStatus(int responseId) {
+  private NodeStatus getNodeStatus(int responseId) throws IOException {
 
     NodeHealthStatus nodeHealthStatus = this.context.getNodeHealthStatus();
     nodeHealthStatus.setHealthReport(healthChecker.getHealthReport());
@@ -352,11 +346,18 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
 
   // Iterate through the NMContext and clone and get all the containers'
   // statuses. If it's a completed container, add into the
-  // recentlyStoppedContainers and previousCompletedContainers collections.
+  // recentlyStoppedContainers collections.
   @VisibleForTesting
-  protected List<ContainerStatus> getContainerStatuses() {
+  protected List<ContainerStatus> getContainerStatuses() throws IOException {
     List<ContainerStatus> containerStatuses = new ArrayList<ContainerStatus>();
     for (Container container : this.context.getContainers().values()) {
+      ContainerId containerId = container.getContainerId();
+      ApplicationId applicationId = container.getContainerId()
+          .getApplicationAttemptId().getApplicationId();
+      if (!this.context.getApplications().containsKey(applicationId)) {
+        context.getContainers().remove(containerId);
+        continue;
+      }
       org.apache.hadoop.yarn.api.records.ContainerStatus containerStatus =
           container.cloneAndGetContainerStatus();
       containerStatuses.add(containerStatus);
@@ -381,10 +382,17 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
   }
 
   // These NMContainerStatus are sent on NM registration and used by YARN only.
-  private List<NMContainerStatus> getNMContainerStatuses() {
+  private List<NMContainerStatus> getNMContainerStatuses() throws IOException {
     List<NMContainerStatus> containerStatuses =
         new ArrayList<NMContainerStatus>();
     for (Container container : this.context.getContainers().values()) {
+      ContainerId containerId = container.getContainerId();
+      ApplicationId applicationId = container.getContainerId()
+          .getApplicationAttemptId().getApplicationId();
+      if (!this.context.getApplications().containsKey(applicationId)) {
+        context.getContainers().remove(containerId);
+        continue;
+      }
       NMContainerStatus status =
           container.getNMContainerStatus();
       containerStatuses.add(status);
@@ -402,26 +410,30 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
 
   @Override
   public void addCompletedContainer(ContainerId containerId) {
-    synchronized (previousCompletedContainers) {
-      previousCompletedContainers.add(containerId);
-    }
     synchronized (recentlyStoppedContainers) {
       removeVeryOldStoppedContainersFromCache();
-      recentlyStoppedContainers.put(containerId,
-        System.currentTimeMillis() + durationToTrackStoppedContainers);
+      if (!recentlyStoppedContainers.containsKey(containerId)) {
+        recentlyStoppedContainers.put(containerId,
+            System.currentTimeMillis() + durationToTrackStoppedContainers);
+      }
     }
   }
 
-  private void removeCompletedContainersFromContext() {
-    synchronized (previousCompletedContainers) {
-      if (!previousCompletedContainers.isEmpty()) {
-        for (ContainerId containerId : previousCompletedContainers) {
-          this.context.getContainers().remove(containerId);
-        }
-        LOG.info("Removed completed containers from NM context: "
-            + previousCompletedContainers);
-        previousCompletedContainers.clear();
-      }
+  @VisibleForTesting
+  @Private
+  public void removeCompletedContainersFromContext(
+      List<ContainerId>containerIds) throws IOException {
+    Set<ContainerId> removedContainers = new HashSet<ContainerId>();
+
+    // If the AM has pulled the completedContainer it can be removed
+    for (ContainerId containerId : containerIds) {
+      context.getContainers().remove(containerId);
+      removedContainers.add(containerId);
+    }
+
+    if (!removedContainers.isEmpty()) {
+      LOG.info("Removed completed containers from NM context: " +
+          removedContainers);
     }
   }
 
@@ -454,7 +466,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
       return recentlyStoppedContainers.containsKey(containerId);
     }
   }
-  
+
   @Override
   public void clearFinishedContainersFromCache() {
     synchronized (recentlyStoppedContainers) {
@@ -472,11 +484,13 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
       while (i.hasNext()) {
         ContainerId cid = i.next();
         if (recentlyStoppedContainers.get(cid) < currentTime) {
-          i.remove();
-          try {
-            context.getNMStateStore().removeContainer(cid);
-          } catch (IOException e) {
-            LOG.error("Unable to remove container " + cid + " in store", e);
+          if (!context.getContainers().containsKey(cid)) {
+            i.remove();
+            try {
+              context.getNMStateStore().removeContainer(cid);
+            } catch (IOException e) {
+              LOG.error("Unable to remove container " + cid + " in store", e);
+            }
           }
         } else {
           break;
@@ -542,7 +556,9 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
             // don't want to remove the completed containers before resync
             // because these completed containers will be reported back to RM
             // when NM re-registers with RM.
-            removeCompletedContainersFromContext();
+            // Only remove the cleanedup containers that are acked
+            removeCompletedContainersFromContext(response
+                  .getFinishedContainersPulledByAM());
 
             lastHeartBeatID = response.getResponseId();
             List<ContainerId> containersToCleanup = response

+ 21 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java

@@ -58,6 +58,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequ
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
@@ -247,6 +248,10 @@ public class TestNodeManagerResync {
                   // put the completed container into the context
                   getNMContext().getContainers().put(
                     testCompleteContainer.getContainerId(), container);
+                  getNMContext().getApplications().put(
+                      testCompleteContainer.getContainerId()
+                          .getApplicationAttemptId().getApplicationId(),
+                      mock(Application.class));
                 } else {
                   // second register contains the completed container info.
                   List<NMContainerStatus> statuses =
@@ -382,9 +387,17 @@ public class TestNodeManagerResync {
             if (containersShouldBePreserved) {
               Assert.assertFalse(containers.isEmpty());
               Assert.assertTrue(containers.containsKey(existingCid));
+              Assert.assertEquals(ContainerState.RUNNING,
+                  containers.get(existingCid)
+                  .cloneAndGetContainerStatus().getState());
             } else {
-              // ensure that containers are empty before restart nodeStatusUpdater
-              Assert.assertTrue(containers.isEmpty());
+              // ensure that containers are empty or are completed before
+              // restart nodeStatusUpdater
+              if (!containers.isEmpty()) {
+                Assert.assertEquals(ContainerState.COMPLETE,
+                    containers.get(existingCid)
+                        .cloneAndGetContainerStatus().getState());
+              }
             }
             super.rebootNodeStatusUpdaterAndRegisterWithRM();
           }
@@ -465,7 +478,12 @@ public class TestNodeManagerResync {
 
         try {
           // ensure that containers are empty before restart nodeStatusUpdater
-          Assert.assertTrue(containers.isEmpty());
+          if (!containers.isEmpty()) {
+            for (Container container: containers.values()) {
+              Assert.assertEquals(ContainerState.COMPLETE,
+                  container.cloneAndGetContainerStatus().getState());
+            }
+          }
           super.rebootNodeStatusUpdaterAndRegisterWithRM();
           // After this point new containers are free to be launched, except
           // containers from previous RM

+ 173 - 61
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java

@@ -58,6 +58,7 @@ import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.client.RMProxy;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -180,7 +181,7 @@ public class TestNodeStatusUpdater {
       Map<ApplicationId, List<ContainerStatus>> map =
           new HashMap<ApplicationId, List<ContainerStatus>>();
       for (ContainerStatus cs : containers) {
-        ApplicationId applicationId = 
+        ApplicationId applicationId =
             cs.getContainerId().getApplicationAttemptId().getApplicationId();
         List<ContainerStatus> appContainers = map.get(applicationId);
         if (appContainers == null) {
@@ -205,10 +206,10 @@ public class TestNodeStatusUpdater {
       nodeStatus.setResponseId(heartBeatID++);
       Map<ApplicationId, List<ContainerStatus>> appToContainers =
           getAppToContainerStatusMap(nodeStatus.getContainersStatuses());
-      
+
       ApplicationId appId1 = ApplicationId.newInstance(0, 1);
       ApplicationId appId2 = ApplicationId.newInstance(0, 2);
-      
+
       if (heartBeatID == 1) {
         Assert.assertEquals(0, nodeStatus.getContainersStatuses().size());
 
@@ -419,7 +420,7 @@ public class TestNodeStatusUpdater {
   }
 
   private class MyNodeManager extends NodeManager {
-    
+
     private MyNodeStatusUpdater3 nodeStatusUpdater;
     @Override
     protected NodeStatusUpdater createNodeStatusUpdater(Context context,
@@ -433,7 +434,7 @@ public class TestNodeStatusUpdater {
       return this.nodeStatusUpdater;
     }
   }
-  
+
   private class MyNodeManager2 extends NodeManager {
     public boolean isStopped = false;
     private NodeStatusUpdater nodeStatusUpdater;
@@ -467,7 +468,7 @@ public class TestNodeStatusUpdater {
       syncBarrier.await(10000, TimeUnit.MILLISECONDS);
     }
   }
-  // 
+  //
   private class MyResourceTracker2 implements ResourceTracker {
     public NodeAction heartBeatNodeAction = NodeAction.NORMAL;
     public NodeAction registerNodeAction = NodeAction.NORMAL;
@@ -478,7 +479,7 @@ public class TestNodeStatusUpdater {
     public RegisterNodeManagerResponse registerNodeManager(
         RegisterNodeManagerRequest request) throws YarnException,
         IOException {
-      
+
       RegisterNodeManagerResponse response = recordFactory
           .newRecordInstance(RegisterNodeManagerResponse.class);
       response.setNodeAction(registerNodeAction );
@@ -493,7 +494,7 @@ public class TestNodeStatusUpdater {
         throws YarnException, IOException {
       NodeStatus nodeStatus = request.getNodeStatus();
       nodeStatus.setResponseId(heartBeatID++);
-      
+
       NodeHeartbeatResponse nhResponse = YarnServerBuilderUtils.
           newNodeHeartbeatResponse(heartBeatID, heartBeatNodeAction, null,
               null, null, null, 1000L);
@@ -501,7 +502,7 @@ public class TestNodeStatusUpdater {
       return nhResponse;
     }
   }
-  
+
   private class MyResourceTracker3 implements ResourceTracker {
     public NodeAction heartBeatNodeAction = NodeAction.NORMAL;
     public NodeAction registerNodeAction = NodeAction.NORMAL;
@@ -513,7 +514,7 @@ public class TestNodeStatusUpdater {
     MyResourceTracker3(Context context) {
       this.context = context;
     }
-    
+
     @Override
     public RegisterNodeManagerResponse registerNodeManager(
         RegisterNodeManagerRequest request) throws YarnException,
@@ -564,6 +565,14 @@ public class TestNodeStatusUpdater {
     public NodeAction registerNodeAction = NodeAction.NORMAL;
     public NodeAction heartBeatNodeAction = NodeAction.NORMAL;
     private Context context;
+    private final ContainerStatus containerStatus2 =
+        createContainerStatus(2, ContainerState.RUNNING);
+    private final ContainerStatus containerStatus3 =
+        createContainerStatus(3, ContainerState.COMPLETE);
+    private final ContainerStatus containerStatus4 =
+        createContainerStatus(4, ContainerState.RUNNING);
+    private final ContainerStatus containerStatus5 =
+        createContainerStatus(5, ContainerState.COMPLETE);
 
     public MyResourceTracker4(Context context) {
       this.context = context;
@@ -583,6 +592,8 @@ public class TestNodeStatusUpdater {
     @Override
     public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
         throws YarnException, IOException {
+      List<ContainerId> finishedContainersPulledByAM = new ArrayList
+          <ContainerId>();
       try {
         if (heartBeatID == 0) {
           Assert.assertEquals(request.getNodeStatus().getContainersStatuses()
@@ -594,10 +605,6 @@ public class TestNodeStatusUpdater {
           Assert.assertEquals(statuses.size(), 2);
           Assert.assertEquals(context.getContainers().size(), 2);
 
-          ContainerStatus containerStatus2 =
-              createContainerStatus(2, ContainerState.RUNNING);
-          ContainerStatus containerStatus3 =
-              createContainerStatus(3, ContainerState.COMPLETE);
           boolean container2Exist = false, container3Exist = false;
           for (ContainerStatus status : statuses) {
             if (status.getContainerId().equals(
@@ -619,23 +626,14 @@ public class TestNodeStatusUpdater {
           // nodeStatusUpdaterRunnable, otherwise nm just shuts down and the
           // test passes.
           throw new YarnRuntimeException("Lost the heartbeat response");
-        } else if (heartBeatID == 2) {
+        } else if (heartBeatID == 2 || heartBeatID == 3) {
           List<ContainerStatus> statuses =
               request.getNodeStatus().getContainersStatuses();
           Assert.assertEquals(statuses.size(), 4);
           Assert.assertEquals(context.getContainers().size(), 4);
 
-          ContainerStatus containerStatus2 =
-              createContainerStatus(2, ContainerState.RUNNING);
-          ContainerStatus containerStatus3 =
-              createContainerStatus(3, ContainerState.COMPLETE);
-          ContainerStatus containerStatus4 =
-              createContainerStatus(4, ContainerState.RUNNING);
-          ContainerStatus containerStatus5 =
-              createContainerStatus(5, ContainerState.COMPLETE);
-
-          boolean container2Exist = false, container3Exist = false, container4Exist =
-              false, container5Exist = false;
+          boolean container2Exist = false, container3Exist = false,
+              container4Exist = false, container5Exist = false;
           for (ContainerStatus status : statuses) {
             if (status.getContainerId().equals(
               containerStatus2.getContainerId())) {
@@ -664,6 +662,24 @@ public class TestNodeStatusUpdater {
           }
           Assert.assertTrue(container2Exist && container3Exist
               && container4Exist && container5Exist);
+
+          if (heartBeatID == 3) {
+            finishedContainersPulledByAM.add(containerStatus3.getContainerId());
+          }
+        } else if (heartBeatID == 4) {
+          List<ContainerStatus> statuses =
+              request.getNodeStatus().getContainersStatuses();
+          Assert.assertEquals(statuses.size(), 3);
+          Assert.assertEquals(context.getContainers().size(), 3);
+
+          boolean container3Exist = false;
+          for (ContainerStatus status : statuses) {
+            if (status.getContainerId().equals(
+                containerStatus3.getContainerId())) {
+              container3Exist = true;
+            }
+          }
+          Assert.assertFalse(container3Exist);
         }
       } catch (AssertionError error) {
         error.printStackTrace();
@@ -676,6 +692,7 @@ public class TestNodeStatusUpdater {
       NodeHeartbeatResponse nhResponse =
           YarnServerBuilderUtils.newNodeHeartbeatResponse(heartBeatID,
             heartBeatNodeAction, null, null, null, null, 1000L);
+      nhResponse.addFinishedContainersPulledByAM(finishedContainersPulledByAM);
       return nhResponse;
     }
   }
@@ -686,7 +703,7 @@ public class TestNodeStatusUpdater {
     public RegisterNodeManagerResponse registerNodeManager(
         RegisterNodeManagerRequest request) throws YarnException,
         IOException {
-      
+
       RegisterNodeManagerResponse response = recordFactory
           .newRecordInstance(RegisterNodeManagerResponse.class);
       response.setNodeAction(registerNodeAction );
@@ -694,7 +711,7 @@ public class TestNodeStatusUpdater {
       response.setNMTokenMasterKey(createMasterKey());
       return response;
     }
-    
+
     @Override
     public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
         throws YarnException, IOException {
@@ -767,11 +784,11 @@ public class TestNodeStatusUpdater {
     lfs.delete(new Path(basedir.getPath()), true);
   }
 
-  @Test(timeout = 90000)                                                      
-  public void testRecentlyFinishedContainers() throws Exception {             
-    NodeManager nm = new NodeManager();                                       
-    YarnConfiguration conf = new YarnConfiguration();                         
-    conf.set(                                                                 
+  @Test(timeout = 90000)
+  public void testRecentlyFinishedContainers() throws Exception {
+    NodeManager nm = new NodeManager();
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.set(
         NodeStatusUpdaterImpl.YARN_NODEMANAGER_DURATION_TO_TRACK_STOPPED_CONTAINERS,
         "10000");                                                             
     nm.init(conf);                                                            
@@ -780,27 +797,112 @@ public class TestNodeStatusUpdater {
     ApplicationId appId = ApplicationId.newInstance(0, 0);                    
     ApplicationAttemptId appAttemptId =                                       
         ApplicationAttemptId.newInstance(appId, 0);                           
-    ContainerId cId = ContainerId.newInstance(appAttemptId, 0);               
-                                                                              
-                                                                              
+    ContainerId cId = ContainerId.newInstance(appAttemptId, 0);
+    nm.getNMContext().getApplications().putIfAbsent(appId,
+        mock(Application.class));
+    nm.getNMContext().getContainers().putIfAbsent(cId, mock(Container.class));
+
     nodeStatusUpdater.addCompletedContainer(cId);
     Assert.assertTrue(nodeStatusUpdater.isContainerRecentlyStopped(cId));     
-                                                                              
+
+    nm.getNMContext().getContainers().remove(cId);
     long time1 = System.currentTimeMillis();                                  
     int waitInterval = 15;                                                    
     while (waitInterval-- > 0                                                 
         && nodeStatusUpdater.isContainerRecentlyStopped(cId)) {               
-      nodeStatusUpdater.removeVeryOldStoppedContainersFromCache();          
+      nodeStatusUpdater.removeVeryOldStoppedContainersFromCache();
       Thread.sleep(1000);                                                     
     }                                                                         
-    long time2 = System.currentTimeMillis();                                  
+    long time2 = System.currentTimeMillis();
     // By this time the container will be removed from cache. need to verify.
-    Assert.assertFalse(nodeStatusUpdater.isContainerRecentlyStopped(cId));    
-    Assert.assertTrue((time2 - time1) >= 10000 && (time2 -time1) <= 250000);  
-  }                                                                           
-                                                                              
+    Assert.assertFalse(nodeStatusUpdater.isContainerRecentlyStopped(cId));
+    Assert.assertTrue((time2 - time1) >= 10000 && (time2 - time1) <= 250000);
+  }
 
+  @Test(timeout = 90000)
+  public void testRemovePreviousCompletedContainersFromContext() throws Exception {
+    NodeManager nm = new NodeManager();
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.set(
+        NodeStatusUpdaterImpl
+            .YARN_NODEMANAGER_DURATION_TO_TRACK_STOPPED_CONTAINERS,
+        "10000");
+    nm.init(conf);
+    NodeStatusUpdaterImpl nodeStatusUpdater =
+        (NodeStatusUpdaterImpl) nm.getNodeStatusUpdater();
+    ApplicationId appId = ApplicationId.newInstance(0, 0);
+    ApplicationAttemptId appAttemptId =
+        ApplicationAttemptId.newInstance(appId, 0);
   
+    ContainerId cId = ContainerId.newInstance(appAttemptId, 1);
+    Token containerToken =
+        BuilderUtils.newContainerToken(cId, "anyHost", 1234, "anyUser",
+            BuilderUtils.newResource(1024, 1), 0, 123,
+            "password".getBytes(), 0);
+    Container anyCompletedContainer = new ContainerImpl(conf, null,
+        null, null, null, null,
+        BuilderUtils.newContainerTokenIdentifier(containerToken)) {
+
+      @Override
+      public ContainerState getCurrentState() {
+        return ContainerState.COMPLETE;
+      }
+    };
+
+    nm.getNMContext().getApplications().putIfAbsent(appId,
+        mock(Application.class));
+    nm.getNMContext().getContainers().put(cId, anyCompletedContainer);
+    Assert.assertEquals(1, nodeStatusUpdater.getContainerStatuses().size());
+
+    List<ContainerId> ackedContainers = new ArrayList<ContainerId>();
+    ackedContainers.add(cId);
+
+    nodeStatusUpdater.removeCompletedContainersFromContext(ackedContainers);
+    Assert.assertTrue(nodeStatusUpdater.getContainerStatuses().isEmpty());
+  }
+
+  @Test
+  public void testCleanedupApplicationContainerCleanup() throws IOException {
+    NodeManager nm = new NodeManager();
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.set(NodeStatusUpdaterImpl
+            .YARN_NODEMANAGER_DURATION_TO_TRACK_STOPPED_CONTAINERS,
+        "1000000");
+    nm.init(conf);
+
+    NodeStatusUpdaterImpl nodeStatusUpdater =
+        (NodeStatusUpdaterImpl) nm.getNodeStatusUpdater();
+    ApplicationId appId = ApplicationId.newInstance(0, 0);
+    ApplicationAttemptId appAttemptId =
+        ApplicationAttemptId.newInstance(appId, 0);
+
+    ContainerId cId = ContainerId.newInstance(appAttemptId, 1);
+    Token containerToken =
+        BuilderUtils.newContainerToken(cId, "anyHost", 1234, "anyUser",
+            BuilderUtils.newResource(1024, 1), 0, 123,
+            "password".getBytes(), 0);
+    Container anyCompletedContainer = new ContainerImpl(conf, null,
+        null, null, null, null,
+        BuilderUtils.newContainerTokenIdentifier(containerToken)) {
+
+      @Override
+      public ContainerState getCurrentState() {
+        return ContainerState.COMPLETE;
+      }
+    };
+
+    nm.getNMContext().getApplications().putIfAbsent(appId,
+        mock(Application.class));
+    nm.getNMContext().getContainers().put(cId, anyCompletedContainer);
+
+    Assert.assertEquals(1, nodeStatusUpdater.getContainerStatuses().size());
+
+    nm.getNMContext().getApplications().remove(appId);
+    nodeStatusUpdater.removeCompletedContainersFromContext(new ArrayList
+        <ContainerId>());
+    Assert.assertEquals(0, nodeStatusUpdater.getContainerStatuses().size());
+  }
+
   @Test
   public void testNMRegistration() throws InterruptedException {
     nm = new NodeManager() {
@@ -860,7 +962,7 @@ public class TestNodeStatusUpdater {
 
     nm.stop();
   }
-  
+
   @Test
   public void testStopReentrant() throws Exception {
     final AtomicInteger numCleanups = new AtomicInteger(0);
@@ -875,7 +977,7 @@ public class TestNodeStatusUpdater {
         myNodeStatusUpdater.resourceTracker = myResourceTracker2;
         return myNodeStatusUpdater;
       }
-      
+
       @Override
       protected ContainerManagerImpl createContainerManager(Context context,
           ContainerExecutor exec, DeletionService del,
@@ -897,7 +999,7 @@ public class TestNodeStatusUpdater {
     YarnConfiguration conf = createNMConfig();
     nm.init(conf);
     nm.start();
-    
+
     int waitCount = 0;
     while (heartBeatID < 1 && waitCount++ != 200) {
       Thread.sleep(500);
@@ -906,7 +1008,7 @@ public class TestNodeStatusUpdater {
 
     // Meanwhile call stop directly as the shutdown hook would
     nm.stop();
-    
+
     // NM takes a while to reach the STOPPED state.
     waitCount = 0;
     while (nm.getServiceState() != STATE.STOPPED && waitCount++ != 20) {
@@ -1172,9 +1274,13 @@ public class TestNodeStatusUpdater {
     nm.start();
 
     int waitCount = 0;
-    while (heartBeatID <= 3 && waitCount++ != 20) {
+    while (heartBeatID <= 4 && waitCount++ != 20) {
       Thread.sleep(500);
     }
+    if (heartBeatID <= 4) {
+      Assert.fail("Failed to get all heartbeats in time, " +
+          "heartbeatID:" + heartBeatID);
+    }
     if(assertionFailedInThread.get()) {
       Assert.fail("ContainerStatus Backup failed");
     }
@@ -1182,7 +1288,7 @@ public class TestNodeStatusUpdater {
   }
 
   @Test(timeout = 200000)
-  public void testNodeStatusUpdaterRetryAndNMShutdown() 
+  public void testNodeStatusUpdaterRetryAndNMShutdown()
       throws Exception {
     final long connectionWaitSecs = 1000;
     final long connectionRetryIntervalMs = 1000;
@@ -1190,7 +1296,7 @@ public class TestNodeStatusUpdater {
     conf.setLong(YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS,
         connectionWaitSecs);
     conf.setLong(YarnConfiguration
-        .RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS,
+            .RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS,
         connectionRetryIntervalMs);
     conf.setLong(YarnConfiguration.NM_SLEEP_DELAY_BEFORE_SIGKILL_MS, 5000);
     conf.setLong(YarnConfiguration.NM_LOG_RETAIN_SECONDS, 1);
@@ -1281,30 +1387,36 @@ public class TestNodeStatusUpdater {
       } else if (heartBeatID == 1) {
         ContainerStatus containerStatus2 =
             createContainerStatus(2, ContainerState.RUNNING);
-        Container container2 = getMockContainer(containerStatus2);
-        containers.put(containerStatus2.getContainerId(), container2);
+        putMockContainer(containerStatus2);
 
         ContainerStatus containerStatus3 =
             createContainerStatus(3, ContainerState.COMPLETE);
-        Container container3 = getMockContainer(containerStatus3);
-        containers.put(containerStatus3.getContainerId(), container3);
+        putMockContainer(containerStatus3);
         return containers;
       } else if (heartBeatID == 2) {
         ContainerStatus containerStatus4 =
             createContainerStatus(4, ContainerState.RUNNING);
-        Container container4 = getMockContainer(containerStatus4);
-        containers.put(containerStatus4.getContainerId(), container4);
+        putMockContainer(containerStatus4);
 
         ContainerStatus containerStatus5 =
             createContainerStatus(5, ContainerState.COMPLETE);
-        Container container5 = getMockContainer(containerStatus5);
-        containers.put(containerStatus5.getContainerId(), container5);
+        putMockContainer(containerStatus5);
+        return containers;
+      } else if (heartBeatID == 3 || heartBeatID == 4) {
         return containers;
       } else {
         containers.clear();
         return containers;
       }
     }
+
+    private void putMockContainer(ContainerStatus containerStatus) {
+      Container container = getMockContainer(containerStatus);
+      containers.put(containerStatus.getContainerId(), container);
+      applications.putIfAbsent(containerStatus.getContainerId()
+          .getApplicationAttemptId().getApplicationId(),
+          mock(Application.class));
+    }
   }
 
   public static ContainerStatus createContainerStatus(int id,
@@ -1345,7 +1457,7 @@ public class TestNodeStatusUpdater {
         throw e;
       }
     }
-    
+
     // the service should be stopped
     Assert.assertEquals("NM state is wrong!", STATE.STOPPED, nm
         .getServiceState());
@@ -1364,7 +1476,7 @@ public class TestNodeStatusUpdater {
     }
     conf.setInt(YarnConfiguration.NM_PMEM_MB, 5 * 1024); // 5GB
     conf.set(YarnConfiguration.NM_ADDRESS, localhostAddress + ":12345");
-    conf.set(YarnConfiguration.NM_LOCALIZER_ADDRESS, localhostAddress + ":12346");  
+    conf.set(YarnConfiguration.NM_LOCALIZER_ADDRESS, localhostAddress + ":12346");
     conf.set(YarnConfiguration.NM_LOG_DIRS, logsDir.getAbsolutePath());
     conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
       remoteLogsDir.getAbsolutePath());
@@ -1372,7 +1484,7 @@ public class TestNodeStatusUpdater {
     conf.setLong(YarnConfiguration.NM_LOG_RETAIN_SECONDS, 1);
     return conf;
   }
-  
+
   private NodeManager getNodeManager(final NodeAction nodeHeartBeatAction) {
     return new NodeManager() {
       @Override

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java

@@ -198,7 +198,7 @@ public class ResourceTrackerService extends AbstractService implements
    */
   @SuppressWarnings("unchecked")
   @VisibleForTesting
-  void handleNMContainerStatus(NMContainerStatus containerStatus) {
+  void handleNMContainerStatus(NMContainerStatus containerStatus, NodeId nodeId) {
     ApplicationAttemptId appAttemptId =
         containerStatus.getContainerId().getApplicationAttemptId();
     RMApp rmApp =
@@ -229,7 +229,8 @@ public class ResourceTrackerService extends AbstractService implements
             containerStatus.getContainerExitStatus());
       // sending master container finished event.
       RMAppAttemptContainerFinishedEvent evt =
-          new RMAppAttemptContainerFinishedEvent(appAttemptId, status);
+          new RMAppAttemptContainerFinishedEvent(appAttemptId, status,
+              nodeId);
       rmContext.getDispatcher().getEventHandler().handle(evt);
     }
   }
@@ -324,7 +325,7 @@ public class ResourceTrackerService extends AbstractService implements
         LOG.info("received container statuses on node manager register :"
             + request.getNMContainerStatuses());
         for (NMContainerStatus status : request.getNMContainerStatuses()) {
-          handleNMContainerStatus(status);
+          handleNMContainerStatus(status, nodeId);
         }
       }
     }

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java

@@ -1181,7 +1181,7 @@ public class RMAppImpl implements RMApp, Recoverable {
       int numberOfFailure = app.getNumFailedAppAttempts();
       if (!app.submissionContext.getUnmanagedAM()
           && numberOfFailure < app.maxAppAttempts) {
-        boolean transferStateFromPreviousAttempt = false;
+        boolean transferStateFromPreviousAttempt;
         RMAppFailedAttemptEvent failedEvent = (RMAppFailedAttemptEvent) event;
         transferStateFromPreviousAttempt =
             failedEvent.getTransferStateFromPreviousAttempt();
@@ -1191,11 +1191,11 @@ public class RMAppImpl implements RMApp, Recoverable {
         // Transfer the state from the previous attempt to the current attempt.
         // Note that the previous failed attempt may still be collecting the
         // container events from the scheduler and update its data structures
-        // before the new attempt is created.
-        if (transferStateFromPreviousAttempt) {
-          ((RMAppAttemptImpl) app.currentAttempt)
-            .transferStateFromPreviousAttempt(oldAttempt);
-        }
+        // before the new attempt is created. We always transferState for
+        // finished containers so that they can be acked to NM,
+        // but when pulling finished container we will check this flag again.
+        ((RMAppAttemptImpl) app.currentAttempt)
+          .transferStateFromPreviousAttempt(oldAttempt);
         return initialState;
       } else {
         if (numberOfFailure >= app.maxAppAttempts) {

+ 20 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt;
 
 import java.util.List;
+import java.util.concurrent.ConcurrentMap;
 
 import javax.crypto.SecretKey;
 
@@ -31,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -120,13 +122,28 @@ public interface RMAppAttempt extends EventHandler<RMAppAttemptEvent> {
   List<ContainerStatus> pullJustFinishedContainers();
 
   /**
-   * Return the list of last set of finished containers. This does not reset the
-   * finished containers.
-   * @return the list of just finished contianers, this does not reset the
+   * Returns a reference to the map of last set of finished containers to the
+   * corresponding node. This does not reset the finished containers.
+   * @return the list of just finished containers, this does not reset the
    * finished containers.
    */
+  ConcurrentMap<NodeId, List<ContainerStatus>>
+      getJustFinishedContainersReference();
+
+  /**
+   * Return the list of last set of finished containers. This does not reset
+   * the finished containers.
+   * @return the list of just finished containers
+   */
   List<ContainerStatus> getJustFinishedContainers();
 
+  /**
+   * The map of conatiners per Node that are already sent to the AM.
+   * @return map of per node list of finished container status sent to AM
+   */
+  ConcurrentMap<NodeId, List<ContainerStatus>>
+      getFinishedContainersSentToAMReference();
+
   /**
    * The container on which the Application Master is running.
    * @return the {@link Container} on which the application master is running.

+ 114 - 19
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java

@@ -24,9 +24,12 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
@@ -52,6 +55,7 @@ import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
@@ -83,6 +87,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAt
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStatusupdateEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeFinishedContainersPulledByAMEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
@@ -129,9 +134,16 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
   private final ApplicationSubmissionContext submissionContext;
   private Token<AMRMTokenIdentifier> amrmToken = null;
   private SecretKey clientTokenMasterKey = null;
-  
-  private List<ContainerStatus> justFinishedContainers =
-    new ArrayList<ContainerStatus>();
+
+  private ConcurrentMap<NodeId, List<ContainerStatus>>
+      justFinishedContainers =
+      new ConcurrentHashMap<NodeId, List<ContainerStatus>>();
+  // Tracks the previous finished containers that are waiting to be
+  // verified as received by the AM. If the AM sends the next allocate
+  // request it implicitly acks this list.
+  private ConcurrentMap<NodeId, List<ContainerStatus>>
+      finishedContainersSentToAM =
+      new ConcurrentHashMap<NodeId, List<ContainerStatus>>();
   private Container masterContainer;
 
   private float progress = 0;
@@ -627,9 +639,27 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     }
   }
 
+  @VisibleForTesting
   @Override
   public List<ContainerStatus> getJustFinishedContainers() {
     this.readLock.lock();
+    try {
+      List<ContainerStatus> returnList = new ArrayList<ContainerStatus>();
+      for (Collection<ContainerStatus> containerStatusList :
+          justFinishedContainers.values()) {
+        returnList.addAll(containerStatusList);
+      }
+      return returnList;
+    } finally {
+      this.readLock.unlock();
+    }
+  }
+
+  @Override
+  public ConcurrentMap<NodeId, List<ContainerStatus>>
+  getJustFinishedContainersReference
+      () {
+    this.readLock.lock();
     try {
       return this.justFinishedContainers;
     } finally {
@@ -637,15 +667,68 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     }
   }
 
+  @Override
+  public ConcurrentMap<NodeId, List<ContainerStatus>>
+  getFinishedContainersSentToAMReference() {
+    this.readLock.lock();
+    try {
+      return this.finishedContainersSentToAM;
+    } finally {
+      this.readLock.unlock();
+    }
+  }
+
   @Override
   public List<ContainerStatus> pullJustFinishedContainers() {
     this.writeLock.lock();
 
     try {
-      List<ContainerStatus> returnList = new ArrayList<ContainerStatus>(
-          this.justFinishedContainers.size());
-      returnList.addAll(this.justFinishedContainers);
-      this.justFinishedContainers.clear();
+      List<ContainerStatus> returnList = new ArrayList<ContainerStatus>();
+
+      // A new allocate means the AM received the previously sent
+      // finishedContainers. We can ack this to NM now
+      for (NodeId nodeId:finishedContainersSentToAM.keySet()) {
+
+        // Clear and get current values
+        List<ContainerStatus> currentSentContainers =
+            finishedContainersSentToAM
+            .put(nodeId, new ArrayList<ContainerStatus>());
+        List<ContainerId> containerIdList = new ArrayList<ContainerId>
+            (currentSentContainers.size());
+        for (ContainerStatus containerStatus:currentSentContainers) {
+          containerIdList.add(containerStatus.getContainerId());
+        }
+        eventHandler.handle(new RMNodeFinishedContainersPulledByAMEvent(
+            nodeId, containerIdList));
+      }
+
+      // Mark every containerStatus as being sent to AM though we may return
+      // only the ones that belong to the current attempt
+      boolean keepContainersAcressAttempts = this.submissionContext
+          .getKeepContainersAcrossApplicationAttempts();
+      for (NodeId nodeId:justFinishedContainers.keySet()) {
+
+        // Clear and get current values
+        List<ContainerStatus> finishedContainers = justFinishedContainers.put
+            (nodeId, new ArrayList<ContainerStatus>());
+
+        if (keepContainersAcressAttempts) {
+          returnList.addAll(finishedContainers);
+        } else {
+          // Filter out containers from previous attempt
+          for (ContainerStatus containerStatus: finishedContainers) {
+            if (containerStatus.getContainerId().getApplicationAttemptId()
+                .equals(this.getAppAttemptId())) {
+              returnList.add(containerStatus);
+            }
+          }
+        }
+
+        finishedContainersSentToAM.putIfAbsent(nodeId, new ArrayList
+              <ContainerStatus>());
+        finishedContainersSentToAM.get(nodeId).addAll(finishedContainers);
+      }
+
       return returnList;
     } finally {
       this.writeLock.unlock();
@@ -732,7 +815,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     }
     setMasterContainer(attemptState.getMasterContainer());
     recoverAppAttemptCredentials(attemptState.getAppAttemptCredentials(),
-      attemptState.getState());
+        attemptState.getState());
     this.recoveredFinalState = attemptState.getState();
     this.originalTrackingUrl = attemptState.getFinalTrackingUrl();
     this.proxiedTrackingUrl = generateProxyUriWithScheme(originalTrackingUrl);
@@ -744,7 +827,9 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
   }
 
   public void transferStateFromPreviousAttempt(RMAppAttempt attempt) {
-    this.justFinishedContainers = attempt.getJustFinishedContainers();
+    this.justFinishedContainers = attempt.getJustFinishedContainersReference();
+    this.finishedContainersSentToAM =
+        attempt.getFinishedContainersSentToAMReference();
   }
 
   private void recoverAppAttemptCredentials(Credentials appAttemptTokens,
@@ -1507,6 +1592,9 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       ContainerStatus containerStatus =
           containerFinishedEvent.getContainerStatus();
 
+      // Add all finished containers so that they can be acked to NM
+      addJustFinishedContainer(appAttempt, containerFinishedEvent);
+
       // Is this container the AmContainer? If the finished container is same as
       // the AMContainer, AppAttempt fails
       if (appAttempt.masterContainer != null
@@ -1519,12 +1607,18 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
         return RMAppAttemptState.FINAL_SAVING;
       }
 
-      // Normal container.Put it in completed containers list
-      appAttempt.justFinishedContainers.add(containerStatus);
       return this.currentState;
     }
   }
 
+  private static void addJustFinishedContainer(RMAppAttemptImpl appAttempt,
+      RMAppAttemptContainerFinishedEvent containerFinishedEvent) {
+    appAttempt.justFinishedContainers.putIfAbsent(containerFinishedEvent
+        .getNodeId(), new ArrayList<ContainerStatus>());
+    appAttempt.justFinishedContainers.get(containerFinishedEvent
+            .getNodeId()).add(containerFinishedEvent.getContainerStatus());
+  }
+
   private static final class ContainerFinishedAtFinalStateTransition
       extends BaseTransition {
     @Override
@@ -1533,10 +1627,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       RMAppAttemptContainerFinishedEvent containerFinishedEvent =
           (RMAppAttemptContainerFinishedEvent) event;
       
-      ContainerStatus containerStatus =
-          containerFinishedEvent.getContainerStatus();
       // Normal container. Add it in completed containers list
-      appAttempt.justFinishedContainers.add(containerStatus);
+      addJustFinishedContainer(appAttempt, containerFinishedEvent);
     }
   }
 
@@ -1569,6 +1661,9 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       ContainerStatus containerStatus =
           containerFinishedEvent.getContainerStatus();
 
+      // Add all finished containers so that they can be acked to NM.
+      addJustFinishedContainer(appAttempt, containerFinishedEvent);
+
       // Is this container the ApplicationMaster container?
       if (appAttempt.masterContainer.getId().equals(
           containerStatus.getContainerId())) {
@@ -1576,8 +1671,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
             appAttempt, containerFinishedEvent);
         return RMAppAttemptState.FINISHED;
       }
-      // Normal container.
-      appAttempt.justFinishedContainers.add(containerStatus);
+
       return RMAppAttemptState.FINISHING;
     }
   }
@@ -1592,6 +1686,9 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       ContainerStatus containerStatus =
           containerFinishedEvent.getContainerStatus();
 
+      // Add all finished containers so that they can be acked to NM.
+      addJustFinishedContainer(appAttempt, containerFinishedEvent);
+
       // If this is the AM container, it means the AM container is finished,
       // but we are not yet acknowledged that the final state has been saved.
       // Thus, we still return FINAL_SAVING state here.
@@ -1611,8 +1708,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
             appAttempt.eventCausingFinalSaving), RMAppAttemptState.FINISHED);
         return;
       }
-      // Normal container.
-      appAttempt.justFinishedContainers.add(containerStatus);
     }
   }
 
@@ -1629,7 +1724,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
         transition(RMAppAttemptImpl appAttempt, RMAppAttemptEvent event) {
       appAttempt.updateInfoOnAMUnregister(amUnregisteredEvent);
       new FinalTransition(RMAppAttemptState.FINISHED).transition(appAttempt,
-        event);
+          event);
     }
   }
 

+ 7 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptContainerFinishedEvent.java

@@ -20,21 +20,27 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event;
 
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
 
 public class RMAppAttemptContainerFinishedEvent extends RMAppAttemptEvent {
 
   private final ContainerStatus containerStatus;
+  private final NodeId nodeId;
 
   public RMAppAttemptContainerFinishedEvent(ApplicationAttemptId appAttemptId, 
-      ContainerStatus containerStatus) {
+      ContainerStatus containerStatus, NodeId nodeId) {
     super(appAttemptId, RMAppAttemptEventType.CONTAINER_FINISHED);
     this.containerStatus = containerStatus;
+    this.nodeId = nodeId;
   }
 
   public ContainerStatus getContainerStatus() {
     return this.containerStatus;
   }
 
+  public NodeId getNodeId() {
+    return this.nodeId;
+  }
 }

+ 7 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java

@@ -78,13 +78,13 @@ public class RMContainerImpl implements RMContainer {
         RMContainerEventType.RECOVER, new ContainerRecoveredTransition())
 
     // Transitions from RESERVED state
-    .addTransition(RMContainerState.RESERVED, RMContainerState.RESERVED, 
+    .addTransition(RMContainerState.RESERVED, RMContainerState.RESERVED,
         RMContainerEventType.RESERVED, new ContainerReservedTransition())
-    .addTransition(RMContainerState.RESERVED, RMContainerState.ALLOCATED, 
+    .addTransition(RMContainerState.RESERVED, RMContainerState.ALLOCATED,
         RMContainerEventType.START, new ContainerStartedTransition())
-    .addTransition(RMContainerState.RESERVED, RMContainerState.KILLED, 
+    .addTransition(RMContainerState.RESERVED, RMContainerState.KILLED,
         RMContainerEventType.KILL) // nothing to do
-    .addTransition(RMContainerState.RESERVED, RMContainerState.RELEASED, 
+    .addTransition(RMContainerState.RESERVED, RMContainerState.RELEASED,
         RMContainerEventType.RELEASED) // nothing to do
        
 
@@ -100,7 +100,7 @@ public class RMContainerImpl implements RMContainer {
     .addTransition(RMContainerState.ACQUIRED, RMContainerState.RUNNING,
         RMContainerEventType.LAUNCHED, new LaunchedTransition())
     .addTransition(RMContainerState.ACQUIRED, RMContainerState.COMPLETED,
-        RMContainerEventType.FINISHED, new ContainerFinishedAtAcquiredState())        
+        RMContainerEventType.FINISHED, new ContainerFinishedAtAcquiredState())
     .addTransition(RMContainerState.ACQUIRED, RMContainerState.RELEASED,
         RMContainerEventType.RELEASED, new KillTransition())
     .addTransition(RMContainerState.ACQUIRED, RMContainerState.EXPIRED,
@@ -495,7 +495,8 @@ public class RMContainerImpl implements RMContainer {
       updateAttemptMetrics(container);
 
       container.eventHandler.handle(new RMAppAttemptContainerFinishedEvent(
-        container.appAttemptId, finishedEvent.getRemoteContainerStatus()));
+        container.appAttemptId, finishedEvent.getRemoteContainerStatus(),
+          container.getAllocatedNode()));
 
       container.rmContext.getRMApplicationHistoryWriter().containerFinished(
         container);

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java

@@ -40,6 +40,9 @@ public enum RMNodeEventType {
   CONTAINER_ALLOCATED,
   CLEANUP_CONTAINER,
 
+  // Source: RMAppAttempt
+  FINISHED_CONTAINERS_PULLED_BY_AM,
+
   // Source: NMLivelinessMonitor
   EXPIRE
 }

+ 41 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeFinishedContainersPulledByAMEvent.java

@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.rmnode;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+
+import java.util.List;
+
+// Happens after an implicit ack from AM that the container completion has
+// been notified successfully to the AM
+public class RMNodeFinishedContainersPulledByAMEvent extends RMNodeEvent {
+
+  private List<ContainerId> containers;
+
+  public RMNodeFinishedContainersPulledByAMEvent(NodeId nodeId,
+      List<ContainerId> containers) {
+    super(nodeId, RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM);
+    this.containers = containers;
+  }
+
+  public List<ContainerId> getContainers() {
+    return this.containers;
+  }
+}

+ 39 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java

@@ -112,6 +112,10 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
   private final Set<ContainerId> containersToClean = new TreeSet<ContainerId>(
       new ContainerIdComparator());
 
+  /* set of containers that were notified to AM about their completion */
+  private final Set<ContainerId> finishedContainersPulledByAM =
+      new HashSet<ContainerId>();
+
   /* the list of applications that have finished and need to be purged */
   private final List<ApplicationId> finishedApplications = new ArrayList<ApplicationId>();
 
@@ -135,7 +139,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
          new UpdateNodeResourceWhenUnusableTransition())
 
      //Transitions from RUNNING state
-     .addTransition(NodeState.RUNNING, 
+     .addTransition(NodeState.RUNNING,
          EnumSet.of(NodeState.RUNNING, NodeState.UNHEALTHY),
          RMNodeEventType.STATUS_UPDATE, new StatusUpdateWhenHealthyTransition())
      .addTransition(NodeState.RUNNING, NodeState.DECOMMISSIONED,
@@ -151,6 +155,9 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
          RMNodeEventType.CLEANUP_APP, new CleanUpAppTransition())
      .addTransition(NodeState.RUNNING, NodeState.RUNNING,
          RMNodeEventType.CLEANUP_CONTAINER, new CleanUpContainerTransition())
+     .addTransition(NodeState.RUNNING, NodeState.RUNNING,
+         RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
+         new FinishedContainersPulledByAMTransition())
      .addTransition(NodeState.RUNNING, NodeState.RUNNING,
          RMNodeEventType.RECONNECTED, new ReconnectNodeTransition())
      .addTransition(NodeState.RUNNING, NodeState.RUNNING,
@@ -158,23 +165,30 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
 
      //Transitions from REBOOTED state
      .addTransition(NodeState.REBOOTED, NodeState.REBOOTED,
-         RMNodeEventType.RESOURCE_UPDATE, 
+         RMNodeEventType.RESOURCE_UPDATE,
          new UpdateNodeResourceWhenUnusableTransition())
          
      //Transitions from DECOMMISSIONED state
      .addTransition(NodeState.DECOMMISSIONED, NodeState.DECOMMISSIONED,
-         RMNodeEventType.RESOURCE_UPDATE, 
+         RMNodeEventType.RESOURCE_UPDATE,
          new UpdateNodeResourceWhenUnusableTransition())
-         
+     .addTransition(NodeState.DECOMMISSIONED, NodeState.DECOMMISSIONED,
+         RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
+         new FinishedContainersPulledByAMTransition())
+
      //Transitions from LOST state
      .addTransition(NodeState.LOST, NodeState.LOST,
-         RMNodeEventType.RESOURCE_UPDATE, 
+         RMNodeEventType.RESOURCE_UPDATE,
          new UpdateNodeResourceWhenUnusableTransition())
+     .addTransition(NodeState.LOST, NodeState.LOST,
+         RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
+         new FinishedContainersPulledByAMTransition())
 
      //Transitions from UNHEALTHY state
-     .addTransition(NodeState.UNHEALTHY, 
+     .addTransition(NodeState.UNHEALTHY,
          EnumSet.of(NodeState.UNHEALTHY, NodeState.RUNNING),
-         RMNodeEventType.STATUS_UPDATE, new StatusUpdateWhenUnHealthyTransition())
+         RMNodeEventType.STATUS_UPDATE,
+         new StatusUpdateWhenUnHealthyTransition())
      .addTransition(NodeState.UNHEALTHY, NodeState.DECOMMISSIONED,
          RMNodeEventType.DECOMMISSION,
          new DeactivateNodeTransition(NodeState.DECOMMISSIONED))
@@ -192,7 +206,10 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
          RMNodeEventType.CLEANUP_CONTAINER, new CleanUpContainerTransition())
      .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
          RMNodeEventType.RESOURCE_UPDATE, new UpdateNodeResourceWhenUnusableTransition())
-         
+     .addTransition(NodeState.UNHEALTHY, NodeState.UNHEALTHY,
+         RMNodeEventType.FINISHED_CONTAINERS_PULLED_BY_AM,
+         new FinishedContainersPulledByAMTransition())
+
      // create the topology tables
      .installTopology(); 
 
@@ -365,8 +382,11 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       response.addAllContainersToCleanup(
           new ArrayList<ContainerId>(this.containersToClean));
       response.addAllApplicationsToCleanup(this.finishedApplications);
+      response.addFinishedContainersPulledByAM(
+          new ArrayList<ContainerId>(this.finishedContainersPulledByAM));
       this.containersToClean.clear();
       this.finishedApplications.clear();
+      this.finishedContainersPulledByAM.clear();
     } finally {
       this.writeLock.unlock();
     }
@@ -652,6 +672,16 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
     }
   }
 
+  public static class FinishedContainersPulledByAMTransition implements
+      SingleArcTransition<RMNodeImpl, RMNodeEvent> {
+
+    @Override
+    public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
+      rmNode.finishedContainersPulledByAM.addAll(((
+          RMNodeFinishedContainersPulledByAMEvent) event).getContainers());
+    }
+  }
+
   public static class DeactivateNodeTransition
     implements SingleArcTransition<RMNodeImpl, RMNodeEvent> {
 
@@ -726,7 +756,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
           new ArrayList<ContainerStatus>();
       for (ContainerStatus remoteContainer : statusEvent.getContainers()) {
         ContainerId containerId = remoteContainer.getContainerId();
-        
+
         // Don't bother with containers already scheduled for cleanup, or for
         // applications already killed. The scheduler doens't need to know any
         // more about this container

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java

@@ -130,7 +130,7 @@ public class AllocationConfiguration {
     queueMaxAMShares = new HashMap<String, Float>();
     userMaxAppsDefault = Integer.MAX_VALUE;
     queueMaxAppsDefault = Integer.MAX_VALUE;
-    queueMaxAMShareDefault = -1.0f;
+    queueMaxAMShareDefault = 0.5f;
     queueAcls = new HashMap<String, Map<QueueACL, AccessControlList>>();
     minSharePreemptionTimeouts = new HashMap<String, Long>();
     fairSharePreemptionTimeouts = new HashMap<String, Long>();

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java

@@ -224,7 +224,7 @@ public class AllocationFileLoaderService extends AbstractService {
         new HashMap<String, Map<QueueACL, AccessControlList>>();
     int userMaxAppsDefault = Integer.MAX_VALUE;
     int queueMaxAppsDefault = Integer.MAX_VALUE;
-    float queueMaxAMShareDefault = -1.0f;
+    float queueMaxAMShareDefault = 0.5f;
     long defaultFairSharePreemptionTimeout = Long.MAX_VALUE;
     long defaultMinSharePreemptionTimeout = Long.MAX_VALUE;
     float defaultFairSharePreemptionThreshold = 0.5f;

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java

@@ -112,7 +112,7 @@ public class FairSchedulerConfiguration extends Configuration {
   protected static final int DEFAULT_WAIT_TIME_BEFORE_KILL = 15000;
 
   /** Whether to assign multiple containers in one check-in. */
-  protected static final String  ASSIGN_MULTIPLE = CONF_PREFIX + "assignmultiple";
+  public static final String  ASSIGN_MULTIPLE = CONF_PREFIX + "assignmultiple";
   protected static final boolean DEFAULT_ASSIGN_MULTIPLE = false;
 
   /** Whether to give more weight to apps requiring many resources. */

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java

@@ -220,9 +220,9 @@ public class FairSchedulerPage extends RmView {
           "    }",
           "  });",
           "  $('#cs').bind('select_node.jstree', function(e, data) {",
-          "    var q = $('.q', data.rslt.obj).first().text();",
-          "    if (q == 'root') q = '';",
-          "    else q = '^' + q.substr(q.lastIndexOf('.') + 1) + '$';",
+          "    var queues = $('.q', data.rslt.obj);",
+          "    var q = '^' + queues.first().text();",
+          "    q += queues.length == 1 ? '$' : '\\\\.';",
           "    $('#apps').dataTable().fnFilter(q, 4, true);",
           "  });",
           "  $('#cs').show();",

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java

@@ -491,7 +491,7 @@ public class TestResourceTrackerService {
             ApplicationAttemptId.newInstance(app.getApplicationId(), 2), 1),
           ContainerState.COMPLETE, Resource.newInstance(1024, 1),
           "Dummy Completed", 0, Priority.newInstance(10), 1234);
-    rm.getResourceTrackerService().handleNMContainerStatus(report);
+    rm.getResourceTrackerService().handleNMContainerStatus(report, null);
     verify(handler, never()).handle((Event) any());
 
     // Case 1.2: Master container is null
@@ -502,7 +502,7 @@ public class TestResourceTrackerService {
           ContainerId.newInstance(currentAttempt.getAppAttemptId(), 0),
           ContainerState.COMPLETE, Resource.newInstance(1024, 1),
           "Dummy Completed", 0, Priority.newInstance(10), 1234);
-    rm.getResourceTrackerService().handleNMContainerStatus(report);
+    rm.getResourceTrackerService().handleNMContainerStatus(report, null);
     verify(handler, never()).handle((Event)any());
 
     // Case 2: Managed AM
@@ -515,7 +515,7 @@ public class TestResourceTrackerService {
           ContainerState.COMPLETE, Resource.newInstance(1024, 1),
           "Dummy Completed", 0, Priority.newInstance(10), 1234);
     try {
-      rm.getResourceTrackerService().handleNMContainerStatus(report);
+      rm.getResourceTrackerService().handleNMContainerStatus(report, null);
     } catch (Exception e) {
       // expected - ignore
     }
@@ -530,7 +530,7 @@ public class TestResourceTrackerService {
       ContainerState.COMPLETE, Resource.newInstance(1024, 1),
       "Dummy Completed", 0, Priority.newInstance(10), 1234);
     try {
-      rm.getResourceTrackerService().handleNMContainerStatus(report);
+      rm.getResourceTrackerService().handleNMContainerStatus(report, null);
     } catch (Exception e) {
       // expected - ignore
     }

+ 23 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java

@@ -60,6 +60,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -369,10 +372,29 @@ public class TestRMApplicationHistoryWriter {
   }
 
   @Test
-  public void testRMWritingMassiveHistory() throws Exception {
+  public void testRMWritingMassiveHistoryForFairSche() throws Exception {
+    //test WritingMassiveHistory for Fair Scheduler.
+    testRMWritingMassiveHistory(true);
+  }
+
+  @Test
+  public void testRMWritingMassiveHistoryForCapacitySche() throws Exception {
+    //test WritingMassiveHistory for Capacity Scheduler.
+    testRMWritingMassiveHistory(false);
+  }
+
+  private void testRMWritingMassiveHistory(boolean isFS) throws Exception {
     // 1. Show RM can run with writing history data
     // 2. Test additional workload of processing history events
     YarnConfiguration conf = new YarnConfiguration();
+    if (isFS) {
+      conf.setBoolean(FairSchedulerConfiguration.ASSIGN_MULTIPLE, true);
+      conf.set("yarn.resourcemanager.scheduler.class",
+          FairScheduler.class.getName());
+    } else {
+      conf.set("yarn.resourcemanager.scheduler.class",
+          CapacityScheduler.class.getName());
+    }
     // don't process history events
     MockRM rm = new MockRM(conf) {
       @Override

+ 17 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java

@@ -98,6 +98,9 @@ public class TestAMRestart {
       Thread.sleep(200);
     }
 
+    ContainerId amContainerId = ContainerId.newInstance(am1
+        .getApplicationAttemptId(), 1);
+
     // launch the 2nd container, for testing running container transferred.
     nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 2, ContainerState.RUNNING);
     ContainerId containerId2 =
@@ -196,11 +199,15 @@ public class TestAMRestart {
     // completed containerId4 is also transferred to the new attempt.
     RMAppAttempt newAttempt =
         app1.getRMAppAttempt(am2.getApplicationAttemptId());
-    // 4 containers finished, acquired/allocated/reserved/completed.
-    waitForContainersToFinish(4, newAttempt);
+    // 4 containers finished, acquired/allocated/reserved/completed + AM
+    // container.
+    waitForContainersToFinish(5, newAttempt);
     boolean container3Exists = false, container4Exists = false, container5Exists =
-        false, container6Exists = false;
+        false, container6Exists = false, amContainerExists = false;
     for(ContainerStatus status :  newAttempt.getJustFinishedContainers()) {
+      if(status.getContainerId().equals(amContainerId)) {
+        amContainerExists = true;
+      }
       if(status.getContainerId().equals(containerId3)) {
         // containerId3 is the container ran by previous attempt but finished by the
         // new attempt.
@@ -220,8 +227,11 @@ public class TestAMRestart {
         container6Exists = true;
       }
     }
-    Assert.assertTrue(container3Exists && container4Exists && container5Exists
-        && container6Exists);
+    Assert.assertTrue(amContainerExists);
+    Assert.assertTrue(container3Exists);
+    Assert.assertTrue(container4Exists);
+    Assert.assertTrue(container5Exists);
+    Assert.assertTrue(container6Exists);
 
     // New SchedulerApplicationAttempt also has the containers info.
     rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
@@ -240,14 +250,14 @@ public class TestAMRestart {
     // all 4 normal containers finished.
     System.out.println("New attempt's just finished containers: "
         + newAttempt.getJustFinishedContainers());
-    waitForContainersToFinish(5, newAttempt);
+    waitForContainersToFinish(6, newAttempt);
     rm1.stop();
   }
 
   private void waitForContainersToFinish(int expectedNum, RMAppAttempt attempt)
       throws InterruptedException {
     int count = 0;
-    while (attempt.getJustFinishedContainers().size() != expectedNum
+    while (attempt.getJustFinishedContainers().size() < expectedNum
         && count < 500) {
       Thread.sleep(100);
       count++;

+ 7 - 5
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

@@ -112,6 +112,13 @@ public class TestProportionalCapacityPreemptionPolicy {
     // report "ideal" preempt
     conf.setFloat(TOTAL_PREEMPTION_PER_ROUND, (float) 1.0);
     conf.setFloat(NATURAL_TERMINATION_FACTOR, (float) 1.0);
+    conf.set(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES,
+        ProportionalCapacityPreemptionPolicy.class.getCanonicalName());
+    conf.setBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true);
+    // FairScheduler doesn't support this test,
+    // Set CapacityScheduler as the scheduler for this test.
+    conf.set("yarn.resourcemanager.scheduler.class",
+        CapacityScheduler.class.getName());
 
     mClock = mock(Clock.class);
     mCS = mock(CapacityScheduler.class);
@@ -441,11 +448,6 @@ public class TestProportionalCapacityPreemptionPolicy {
   
   @Test
   public void testPolicyInitializeAfterSchedulerInitialized() {
-    Configuration conf = new Configuration();
-    conf.set(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES,
-        ProportionalCapacityPreemptionPolicy.class.getCanonicalName());
-    conf.setBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true);
-    
     @SuppressWarnings("resource")
     MockRM rm = new MockRM(conf);
     rm.init(conf);

+ 119 - 24
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java

@@ -28,6 +28,7 @@ import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
@@ -35,6 +36,7 @@ import static org.mockito.Mockito.when;
 
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -91,6 +93,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAt
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 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.rmnode.RMNodeEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeFinishedContainersPulledByAMEvent;
+
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
@@ -151,6 +158,7 @@ public class TestRMAppAttemptTransitions {
   private NMTokenSecretManagerInRM nmTokenManager =
       spy(new NMTokenSecretManagerInRM(conf));
   private boolean transferStateFromPreviousAttempt = false;
+  private EventHandler<RMNodeEvent> rmnodeEventHandler;
 
   private final class TestApplicationAttemptEventDispatcher implements
       EventHandler<RMAppAttemptEvent> {
@@ -203,7 +211,7 @@ public class TestRMAppAttemptTransitions {
       applicationMasterLauncher.handle(event);
     }
   }
-  
+
   private static int appId = 1;
   
   private ApplicationSubmissionContext submissionContext = null;
@@ -268,6 +276,9 @@ public class TestRMAppAttemptTransitions {
     rmDispatcher.register(AMLauncherEventType.class, 
         new TestAMLauncherEventDispatcher());
 
+    rmnodeEventHandler = mock(RMNodeImpl.class);
+    rmDispatcher.register(RMNodeEventType.class, rmnodeEventHandler);
+
     rmDispatcher.init(conf);
     rmDispatcher.start();
     
@@ -575,6 +586,8 @@ public class TestRMAppAttemptTransitions {
     }
     assertEquals(finishedContainerCount, applicationAttempt
         .getJustFinishedContainers().size());
+    Assert.assertEquals(0, getFinishedContainersSentToAM(applicationAttempt)
+        .size());
     assertEquals(container, applicationAttempt.getMasterContainer());
     assertEquals(finalStatus, applicationAttempt.getFinalApplicationStatus());
     verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
@@ -704,7 +717,8 @@ public class TestRMAppAttemptTransitions {
     application.handle(new RMAppRunningOnNodeEvent(application.getApplicationId(),
         container.getNodeId()));
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
-        applicationAttempt.getAppAttemptId(), mock(ContainerStatus.class)));
+        applicationAttempt.getAppAttemptId(), mock(ContainerStatus.class),
+        container.getNodeId()));
     // complete AM
     String diagnostics = "Successful";
     FinalApplicationStatus finalStatus = FinalApplicationStatus.SUCCEEDED;
@@ -752,10 +766,11 @@ public class TestRMAppAttemptTransitions {
     when(appResUsgRpt.getMemorySeconds()).thenReturn(223456L);
     when(appResUsgRpt.getVcoreSeconds()).thenReturn(75544L);
     sendAttemptUpdateSavedEvent(applicationAttempt);
+    NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
         attemptId, 
         ContainerStatus.newInstance(
-            amContainer.getId(), ContainerState.COMPLETE, "", 0)));
+            amContainer.getId(), ContainerState.COMPLETE, "", 0), anyNodeId));
 
     when(scheduler.getSchedulerAppInfo(eq(attemptId))).thenReturn(null);
 
@@ -857,8 +872,9 @@ public class TestRMAppAttemptTransitions {
             SchedulerUtils.LOST_CONTAINER);
     // send CONTAINER_FINISHED event at SCHEDULED state,
     // The state should be FINAL_SAVING with previous state SCHEDULED
+    NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
-        applicationAttempt.getAppAttemptId(), cs));
+        applicationAttempt.getAppAttemptId(), cs, anyNodeId));
     // createApplicationAttemptState will return previous state (SCHEDULED),
     // if the current state is FINAL_SAVING.
     assertEquals(YarnApplicationAttemptState.SCHEDULED,
@@ -904,8 +920,9 @@ public class TestRMAppAttemptTransitions {
     ContainerStatus cs =
         BuilderUtils.newContainerStatus(amContainer.getId(),
           ContainerState.COMPLETE, containerDiagMsg, exitCode);
+    NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
-      applicationAttempt.getAppAttemptId(), cs));
+      applicationAttempt.getAppAttemptId(), cs, anyNodeId));
     assertEquals(YarnApplicationAttemptState.ALLOCATED,
         applicationAttempt.createApplicationAttemptState());
     sendAttemptUpdateSavedEvent(applicationAttempt);
@@ -928,16 +945,17 @@ public class TestRMAppAttemptTransitions {
     ContainerStatus cs = BuilderUtils.newContainerStatus(amContainer.getId(),
         ContainerState.COMPLETE, containerDiagMsg, exitCode);
     ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId();
+    NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
-        appAttemptId, cs));
+        appAttemptId, cs, anyNodeId));
 
     // ignored ContainerFinished and Expire at FinalSaving if we were supposed
     // to Failed state.
     assertEquals(RMAppAttemptState.FINAL_SAVING,
-      applicationAttempt.getAppAttemptState()); 
+      applicationAttempt.getAppAttemptState());
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
       applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus(
-        amContainer.getId(), ContainerState.COMPLETE, "", 0)));
+        amContainer.getId(), ContainerState.COMPLETE, "", 0), anyNodeId));
     applicationAttempt.handle(new RMAppAttemptEvent(
       applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE));
     assertEquals(RMAppAttemptState.FINAL_SAVING,
@@ -947,7 +965,7 @@ public class TestRMAppAttemptTransitions {
     sendAttemptUpdateSavedEvent(applicationAttempt);
     assertEquals(RMAppAttemptState.FAILED,
         applicationAttempt.getAppAttemptState());
-    assertEquals(0,applicationAttempt.getJustFinishedContainers().size());
+    assertEquals(2, applicationAttempt.getJustFinishedContainers().size());
     assertEquals(amContainer, applicationAttempt.getMasterContainer());
     assertEquals(0, application.getRanNodes().size());
     String rmAppPageUrl = pjoin(RM_WEBAPP_ADDR, "cluster", "app",
@@ -971,10 +989,11 @@ public class TestRMAppAttemptTransitions {
     // ignored ContainerFinished and Expire at FinalSaving if we were supposed
     // to Killed state.
     assertEquals(RMAppAttemptState.FINAL_SAVING,
-      applicationAttempt.getAppAttemptState()); 
+      applicationAttempt.getAppAttemptState());
+    NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
       applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus(
-        amContainer.getId(), ContainerState.COMPLETE, "", 0)));
+        amContainer.getId(), ContainerState.COMPLETE, "", 0), anyNodeId));
     applicationAttempt.handle(new RMAppAttemptEvent(
       applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE));
     assertEquals(RMAppAttemptState.FINAL_SAVING,
@@ -984,7 +1003,7 @@ public class TestRMAppAttemptTransitions {
     sendAttemptUpdateSavedEvent(applicationAttempt);
     assertEquals(RMAppAttemptState.KILLED,
         applicationAttempt.getAppAttemptState());
-    assertEquals(0,applicationAttempt.getJustFinishedContainers().size());
+    assertEquals(1,applicationAttempt.getJustFinishedContainers().size());
     assertEquals(amContainer, applicationAttempt.getMasterContainer());
     assertEquals(0, application.getRanNodes().size());
     String rmAppPageUrl = pjoin(RM_WEBAPP_ADDR, "cluster", "app",
@@ -1144,13 +1163,14 @@ public class TestRMAppAttemptTransitions {
     unregisterApplicationAttempt(amContainer, finalStatus, trackingUrl,
         diagnostics);
     // container must be AM container to move from FINISHING to FINISHED
+    NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(
         new RMAppAttemptContainerFinishedEvent(
             applicationAttempt.getAppAttemptId(),
             BuilderUtils.newContainerStatus(
                 BuilderUtils.newContainerId(
                     applicationAttempt.getAppAttemptId(), 42),
-                ContainerState.COMPLETE, "", 0)));
+                ContainerState.COMPLETE, "", 0), anyNodeId));
     testAppAttemptFinishingState(amContainer, finalStatus, trackingUrl,
         diagnostics);
   }
@@ -1165,13 +1185,14 @@ public class TestRMAppAttemptTransitions {
     String diagnostics = "Successful";
     unregisterApplicationAttempt(amContainer, finalStatus, trackingUrl,
         diagnostics);
+    NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(
         new RMAppAttemptContainerFinishedEvent(
             applicationAttempt.getAppAttemptId(),
             BuilderUtils.newContainerStatus(amContainer.getId(),
-                ContainerState.COMPLETE, "", 0)));
+                ContainerState.COMPLETE, "", 0), anyNodeId));
     testAppAttemptFinishedState(amContainer, finalStatus, trackingUrl,
-        diagnostics, 0, false);
+        diagnostics, 1, false);
   }
 
   // While attempt is at FINAL_SAVING, Contaienr_Finished event may come before
@@ -1195,15 +1216,16 @@ public class TestRMAppAttemptTransitions {
     assertEquals(YarnApplicationAttemptState.RUNNING,
         applicationAttempt.createApplicationAttemptState());
     // Container_finished event comes before Attempt_Saved event.
+    NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
       applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus(
-        amContainer.getId(), ContainerState.COMPLETE, "", 0)));
+        amContainer.getId(), ContainerState.COMPLETE, "", 0), anyNodeId));
     assertEquals(RMAppAttemptState.FINAL_SAVING,
       applicationAttempt.getAppAttemptState());
     // send attempt_saved
     sendAttemptUpdateSavedEvent(applicationAttempt);
     testAppAttemptFinishedState(amContainer, finalStatus, trackingUrl,
-      diagnostics, 0, false);
+      diagnostics, 1, false);
   }
 
   // While attempt is at FINAL_SAVING, Expire event may come before
@@ -1235,6 +1257,71 @@ public class TestRMAppAttemptTransitions {
       diagnostics, 0, false);
   }
 
+  @Test
+  public void testFinishedContainer() {
+    Container amContainer = allocateApplicationAttempt();
+    launchApplicationAttempt(amContainer);
+    runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
+
+    // Complete one container
+    ContainerId containerId1 = BuilderUtils.newContainerId(applicationAttempt
+        .getAppAttemptId(), 2);
+    Container container1 = mock(Container.class);
+    ContainerStatus containerStatus1 = mock(ContainerStatus.class);
+    when(container1.getId()).thenReturn(
+        containerId1);
+    when(containerStatus1.getContainerId()).thenReturn(containerId1);
+    when(container1.getNodeId()).thenReturn(NodeId.newInstance("host", 1234));
+
+    application.handle(new RMAppRunningOnNodeEvent(application
+        .getApplicationId(),
+        container1.getNodeId()));
+    applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
+        applicationAttempt.getAppAttemptId(), containerStatus1,
+        container1.getNodeId()));
+
+    ArgumentCaptor<RMNodeFinishedContainersPulledByAMEvent> captor =
+        ArgumentCaptor.forClass(RMNodeFinishedContainersPulledByAMEvent.class);
+
+    // Verify justFinishedContainers
+    Assert.assertEquals(1, applicationAttempt.getJustFinishedContainers()
+        .size());
+    Assert.assertEquals(container1.getId(), applicationAttempt
+        .getJustFinishedContainers().get(0).getContainerId());
+    Assert.assertEquals(0, getFinishedContainersSentToAM(applicationAttempt)
+        .size());
+
+    // Verify finishedContainersSentToAM gets container after pull
+    List<ContainerStatus> containerStatuses = applicationAttempt
+        .pullJustFinishedContainers();
+    Assert.assertEquals(1, containerStatuses.size());
+    Mockito.verify(rmnodeEventHandler, never()).handle(Mockito
+        .any(RMNodeEvent.class));
+    Assert.assertTrue(applicationAttempt.getJustFinishedContainers().isEmpty());
+    Assert.assertEquals(1, getFinishedContainersSentToAM(applicationAttempt)
+        .size());
+
+    // Verify container is acked to NM via the RMNodeEvent after second pull
+    containerStatuses = applicationAttempt.pullJustFinishedContainers();
+    Assert.assertEquals(0, containerStatuses.size());
+    Mockito.verify(rmnodeEventHandler).handle(captor.capture());
+    Assert.assertEquals(container1.getId(), captor.getValue().getContainers()
+        .get(0));
+    Assert.assertTrue(applicationAttempt.getJustFinishedContainers().isEmpty());
+    Assert.assertEquals(0, getFinishedContainersSentToAM(applicationAttempt)
+        .size());
+  }
+
+  private static List<ContainerStatus> getFinishedContainersSentToAM(
+      RMAppAttempt applicationAttempt) {
+    List<ContainerStatus> containers = new ArrayList<ContainerStatus>();
+    for (List<ContainerStatus> containerStatuses: applicationAttempt
+        .getFinishedContainersSentToAMReference().values()) {
+      containers.addAll(containerStatuses);
+    }
+    return containers;
+  }
+
   // this is to test user can get client tokens only after the client token
   // master key is saved in the state store and also registered in
   // ClientTokenSecretManager
@@ -1281,8 +1368,9 @@ public class TestRMAppAttemptTransitions {
         ContainerStatus.newInstance(amContainer.getId(),
           ContainerState.COMPLETE, "some error", 123);
     ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId();
+    NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
-      appAttemptId, cs1));
+      appAttemptId, cs1, anyNodeId));
     assertEquals(YarnApplicationAttemptState.RUNNING,
         applicationAttempt.createApplicationAttemptState());
     sendAttemptUpdateSavedEvent(applicationAttempt);
@@ -1293,15 +1381,21 @@ public class TestRMAppAttemptTransitions {
     verifyApplicationAttemptFinished(RMAppAttemptState.FAILED);
 
     // failed attempt captured the container finished event.
-    assertEquals(0, applicationAttempt.getJustFinishedContainers().size());
+    assertEquals(1, applicationAttempt.getJustFinishedContainers().size());
     ContainerStatus cs2 =
         ContainerStatus.newInstance(ContainerId.newInstance(appAttemptId, 2),
           ContainerState.COMPLETE, "", 0);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
-      appAttemptId, cs2));
-    assertEquals(1, applicationAttempt.getJustFinishedContainers().size());
-    assertEquals(cs2.getContainerId(), applicationAttempt
-      .getJustFinishedContainers().get(0).getContainerId());
+      appAttemptId, cs2, anyNodeId));
+    assertEquals(2, applicationAttempt.getJustFinishedContainers().size());
+    boolean found = false;
+    for (ContainerStatus containerStatus:applicationAttempt
+        .getJustFinishedContainers()) {
+      if (cs2.getContainerId().equals(containerStatus.getContainerId())) {
+        found = true;
+      }
+    }
+    assertTrue(found);
   }
 
 
@@ -1322,8 +1416,9 @@ public class TestRMAppAttemptTransitions {
         ContainerStatus.newInstance(amContainer.getId(),
           ContainerState.COMPLETE, "some error", 123);
     ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId();
+    NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
-      appAttemptId, cs1));
+      appAttemptId, cs1, anyNodeId));
     assertEquals(YarnApplicationAttemptState.RUNNING,
         applicationAttempt.createApplicationAttemptState());
     sendAttemptUpdateSavedEvent(applicationAttempt);

+ 16 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java

@@ -3128,7 +3128,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     out.println("<queue name=\"queue1\">");
     out.println("</queue>");
     out.println("<queue name=\"queue2\">");
-    out.println("<maxAMShare>1.0</maxAMShare>");
+    out.println("<maxAMShare>0.4</maxAMShare>");
     out.println("</queue>");
     out.println("<queue name=\"queue3\">");
     out.println("</queue>");
@@ -3172,40 +3172,42 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     assertEquals("Queue queue5's fair share should be 0", 0, queue5
         .getFairShare().getMemory());
 
-    List<String> queues = Arrays.asList("root.default", "root.queue3",
-        "root.queue4", "root.queue5");
+    List<String> queues = Arrays.asList("root.queue3", "root.queue4",
+        "root.queue5");
     for (String queue : queues) {
       createSchedulingRequest(1 * 1024, queue, "user1");
       scheduler.update();
       scheduler.handle(updateEvent);
     }
 
-    Resource amResource1 = Resource.newInstance(2048, 1);
+    Resource amResource1 = Resource.newInstance(1024, 1);
     int amPriority = RMAppAttemptImpl.AM_CONTAINER_PRIORITY.getPriority();
 
-    // Exceeds queue limit, but default maxAMShare is -1.0 so it doesn't matter
+    // The fair share is 2048 MB, and the default maxAMShare is 0.5f,
+    // so the AM is accepted.
     ApplicationAttemptId attId1 = createAppAttemptId(1, 1);
     createApplicationWithAMResource(attId1, "queue1", "test1", amResource1);
-    createSchedulingRequestExistingApplication(2048, 1, amPriority, attId1);
+    createSchedulingRequestExistingApplication(1024, 1, amPriority, attId1);
     FSAppAttempt app1 = scheduler.getSchedulerApp(attId1);
     scheduler.update();
     scheduler.handle(updateEvent);
-    assertEquals("Application1's AM requests 2048 MB memory",
-        2048, app1.getAMResource().getMemory());
+    assertEquals("Application1's AM requests 1024 MB memory",
+        1024, app1.getAMResource().getMemory());
     assertEquals("Application1's AM should be running",
         1, app1.getLiveContainers().size());
-    assertEquals("Queue1's AM resource usage should be 2048 MB memory",
-        2048, queue1.getAmResourceUsage().getMemory());
+    assertEquals("Queue1's AM resource usage should be 1024 MB memory",
+        1024, queue1.getAmResourceUsage().getMemory());
 
-    // Exceeds queue limit, and maxAMShare is 1.0
+    // Now the fair share is 1639 MB, and the maxAMShare is 0.4f,
+    // so the AM is not accepted.
     ApplicationAttemptId attId2 = createAppAttemptId(2, 1);
     createApplicationWithAMResource(attId2, "queue2", "test1", amResource1);
-    createSchedulingRequestExistingApplication(2048, 1, amPriority, attId2);
+    createSchedulingRequestExistingApplication(1024, 1, amPriority, attId2);
     FSAppAttempt app2 = scheduler.getSchedulerApp(attId2);
     scheduler.update();
     scheduler.handle(updateEvent);
-    assertEquals("Application2's AM requests 2048 MB memory",
-        2048, app2.getAMResource().getMemory());
+    assertEquals("Application2's AM requests 1024 MB memory",
+        1024, app2.getAMResource().getMemory());
     assertEquals("Application2's AM should not be running",
         0, app2.getLiveContainers().size());
     assertEquals("Queue2's AM resource usage should be 0 MB memory",

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.java

@@ -161,7 +161,7 @@ public class TestAMRMTokens {
           .getEventHandler()
           .handle(
               new RMAppAttemptContainerFinishedEvent(applicationAttemptId,
-                  containerStatus));
+                  containerStatus, nm1.getNodeId()));
 
       // Make sure the RMAppAttempt is at Finished State.
       // Both AMRMToken and ClientToAMToken have been removed.

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm

@@ -246,8 +246,8 @@ Allocation file format
    * maxAMShare: limit the fraction of the queue's fair share that can be used
      to run application masters. This property can only be used for leaf queues.
      For example, if set to 1.0f, then AMs in the leaf queue can take up to 100%
-     of both the memory and CPU fair share. The default value is -1.0f, which
-     means that this check is disabled.
+     of both the memory and CPU fair share. The value of -1.0f will disable
+     this feature and the amShare will not be checked. The default value is 0.5f.
 
    * weight: to share the cluster non-proportionally with other queues. Weights
      default to 1, and a queue with weight 2 should receive approximately twice