浏览代码

Merging r1547658 through r1548328 from trunk to branch HDFS-2832

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1548329 13f79535-47bb-0310-9956-ffa450edef68
Arpit Agarwal 11 年之前
父节点
当前提交
00718c2ffa
共有 88 个文件被更改,包括 3890 次插入1577 次删除
  1. 24 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 13 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpConfig.java
  3. 1 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
  4. 1 3
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  5. 4 0
      hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm
  6. 1 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java
  7. 65 22
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/IdUserGroup.java
  8. 56 0
      hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/nfs3/TestIdUserGroup.java
  9. 15 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  10. 4 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  11. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  12. 54 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  13. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
  14. 41 22
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirective.java
  15. 56 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
  16. 35 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java
  17. 53 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
  18. 26 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java
  19. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  20. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
  21. 10 35
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  22. 20 64
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  23. 8 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  24. 44 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
  25. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
  26. 49 22
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  27. 44 57
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java
  28. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
  29. 14 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
  30. 52 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java
  31. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
  32. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  33. 61 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
  34. 6 51
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  35. 95 53
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
  36. 5 48
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  37. 2 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  38. 6 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  39. 36 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
  40. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  41. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  42. 13 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  43. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java
  44. 130 40
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
  45. 36 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
  46. 127 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeHttpServer.java
  47. 3 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
  48. 92 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
  49. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHttpsFileSystem.java
  50. 58 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCacheAdminConf.xml
  51. 6 0
      hadoop-mapreduce-project/CHANGES.txt
  52. 17 23
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
  53. 12 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java
  54. 68 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
  55. 1 1
      hadoop-project/pom.xml
  56. 5 0
      hadoop-tools/hadoop-distcp/pom.xml
  57. 11 6
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/SimpleCopyListing.java
  58. 6 1
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java
  59. 28 0
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestCopyListing.java
  60. 59 0
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestRetriableFileCopyCommand.java
  61. 6 0
      hadoop-yarn-project/CHANGES.txt
  62. 78 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceDecrease.java
  63. 84 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncrease.java
  64. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncreaseRequest.java
  65. 16 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
  66. 136 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceDecreasePBImpl.java
  67. 171 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreasePBImpl.java
  68. 141 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreaseRequestPBImpl.java
  69. 66 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceDecrease.java
  70. 74 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncrease.java
  71. 68 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncreaseRequest.java
  72. 229 0
      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
  73. 398 0
      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
  74. 4 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
  75. 6 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
  76. 8 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
  77. 65 29
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
  78. 2 33
      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
  79. 13 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/MaxRunningAppsEnforcer.java
  80. 36 504
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
  81. 32 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueuePlacementPolicy.java
  82. 1 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
  83. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java
  84. 432 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java
  85. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java
  86. 113 342
      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
  87. 0 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java
  88. 9 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestMaxRunningAppsEnforcer.java

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

@@ -393,6 +393,8 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10127. Add ipc.client.connect.retry.interval to control the frequency
     of connection retries (Karthik Kambatla via Sandy Ryza)
 
+    HADOOP-10102. Update commons IO from 2.1 to 2.4 (Akira Ajisaka via stevel)
+
   OPTIMIZATIONS
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
@@ -458,6 +460,11 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10135 writes to swift fs over partition size leave temp files and
     empty output file (David Dobbins via stevel)
 
+    HADOOP-10129. Distcp may succeed when it fails (daryn)
+
+    HADOOP-10058. TestMetricsSystemImpl#testInitFirstVerifyStopInvokedImmediately
+    fails on trunk (Chen He via jeagles)
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -525,6 +532,9 @@ Release 2.3.0 - UNRELEASED
     HDFS-5560. Trash configuration log statements prints incorrect units.
     (Josh Elser via Andrew Wang)
 
+    HADOOP-10081. Client.setupIOStreams can leak socket resources on exception
+    or error (Tsuyoshi OZAWA via jlowe)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES
@@ -2332,6 +2342,20 @@ Release 2.0.0-alpha - 05-23-2012
     HADOOP-7606. Upgrade Jackson to version 1.7.1 to match the version required
     by Jersey (Alejandro Abdelnur via atm)
 
+Release 0.23.11 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+    
+  IMPROVEMENTS
+    
+  OPTIMIZATIONS
+
+  BUG FIXES
+
+    HADOOP-10129. Distcp may succeed when it fails (daryn)
+
 Release 0.23.10 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 13 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpConfig.java

@@ -31,15 +31,25 @@ public class HttpConfig {
   private static Policy policy;
   public enum Policy {
     HTTP_ONLY,
-    HTTPS_ONLY;
+    HTTPS_ONLY,
+    HTTP_AND_HTTPS;
 
     public static Policy fromString(String value) {
-      if (value.equalsIgnoreCase(CommonConfigurationKeysPublic
-              .HTTP_POLICY_HTTPS_ONLY)) {
+      if (HTTPS_ONLY.name().equalsIgnoreCase(value)) {
         return HTTPS_ONLY;
+      } else if (HTTP_AND_HTTPS.name().equalsIgnoreCase(value)) {
+        return HTTP_AND_HTTPS;
       }
       return HTTP_ONLY;
     }
+
+    public boolean isHttpEnabled() {
+      return this == HTTP_ONLY || this == HTTP_AND_HTTPS;
+    }
+
+    public boolean isHttpsEnabled() {
+      return this == HTTPS_ONLY || this == HTTP_AND_HTTPS;
+    }
   }
 
   static {

+ 1 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java

@@ -1158,6 +1158,7 @@ public class Client {
         // cleanup calls
         cleanupCalls();
       }
+      closeConnection();
       if (LOG.isDebugEnabled())
         LOG.debug(getName() + ": closed");
     }

+ 1 - 3
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -1133,9 +1133,7 @@
   <name>hadoop.ssl.enabled</name>
   <value>false</value>
   <description>
-    Whether to use SSL for the HTTP endpoints. If set to true, the
-    NameNode, DataNode, ResourceManager, NodeManager, HistoryServer and
-    MapReduceAppMaster web UIs will be served over HTTPS instead HTTP.
+    Deprecated. Use dfs.http.policy and yarn.http.policy instead.
   </description>
 </property>
 

+ 4 - 0
hadoop-common-project/hadoop-common/src/site/apt/ClusterSetup.apt.vm

@@ -754,6 +754,10 @@ KVNO Timestamp         Principal
 | | | Enable HDFS block access tokens for secure operations. |
 *-------------------------+-------------------------+------------------------+
 | <<<dfs.https.enable>>> | <true> | |
+| | | This value is deprecated. Use dfs.http.policy |
+*-------------------------+-------------------------+------------------------+
+| <<<dfs.http.policy>>> | <HTTP_ONLY> or <HTTPS_ONLY> or <HTTP_AND_HTTPS> | |
+| | | HTTPS_ONLY turns off http access |
 *-------------------------+-------------------------+------------------------+
 | <<<dfs.namenode.https-address>>> | <nn_host_fqdn:50470> | |
 *-------------------------+-------------------------+------------------------+

+ 1 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java

@@ -85,6 +85,7 @@ public class TestMetricsSystemImpl {
   }
 
   @Test public void testInitFirstVerifyStopInvokedImmediately() throws Exception {
+    DefaultMetricsSystem.shutdown();
     new ConfigBuilder().add("*.period", 8)
         //.add("test.sink.plugin.urls", getPluginUrlsAsString())
         .add("test.sink.test.class", TestSink.class.getName())

+ 65 - 22
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/IdUserGroup.java

@@ -24,6 +24,7 @@ import java.io.InputStreamReader;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.BiMap;
 import com.google.common.collect.HashBiMap;
 
@@ -44,13 +45,21 @@ public class IdUserGroup {
   // Do update every 15 minutes
   final static long TIMEOUT = 15 * 60 * 1000; // ms
 
-  // Maps for id to name map. Guarded by this object monitor lock */
+  // Maps for id to name map. Guarded by this object monitor lock
   private BiMap<Integer, String> uidNameMap = HashBiMap.create();
   private BiMap<Integer, String> gidNameMap = HashBiMap.create();
 
   private long lastUpdateTime = 0; // Last time maps were updated
 
-  public IdUserGroup() {
+  static public class DuplicateNameOrIdException extends IOException {
+    private static final long serialVersionUID = 1L;
+
+    public DuplicateNameOrIdException(String msg) {
+      super(msg);
+    }
+  }
+  
+  public IdUserGroup() throws IOException {
     updateMaps();
   }
 
@@ -58,18 +67,34 @@ public class IdUserGroup {
     return lastUpdateTime - System.currentTimeMillis() > TIMEOUT;
   }
 
+  // If can't update the maps, will keep using the old ones
   private void checkAndUpdateMaps() {
     if (isExpired()) {
       LOG.info("Update cache now");
-      updateMaps();
+      try {
+        updateMaps();
+      } catch (IOException e) {
+        LOG.error("Can't update the maps. Will use the old ones,"
+            + " which can potentially cause problem.", e);
+      }
     }
   }
 
+  private static final String DUPLICATE_NAME_ID_DEBUG_INFO = "NFS gateway can't start with duplicate name or id on the host system.\n"
+      + "This is because HDFS (non-kerberos cluster) uses name as the only way to identify a user or group.\n"
+      + "The host system with duplicated user/group name or id might work fine most of the time by itself.\n"
+      + "However when NFS gateway talks to HDFS, HDFS accepts only user and group name.\n"
+      + "Therefore, same name means the same user or same group. To find the duplicated names/ids, one can do:\n"
+      + "<getent passwd | cut -d: -f1,3> and <getent group | cut -d: -f1,3> on Linux systms,\n"
+      + "<dscl . -list /Users UniqueID> and <dscl . -list /Groups PrimaryGroupID> on MacOS.";
+  
   /**
    * Get the whole list of users and groups and save them in the maps.
+   * @throws IOException 
    */
-  private void updateMapInternal(BiMap<Integer, String> map, String name,
-      String command, String regex) throws IOException {
+  @VisibleForTesting
+  public static void updateMapInternal(BiMap<Integer, String> map, String mapName,
+      String command, String regex) throws IOException  {
     BufferedReader br = null;
     try {
       Process process = Runtime.getRuntime().exec(
@@ -79,15 +104,31 @@ public class IdUserGroup {
       while ((line = br.readLine()) != null) {
         String[] nameId = line.split(regex);
         if ((nameId == null) || (nameId.length != 2)) {
-          throw new IOException("Can't parse " + name + " list entry:" + line);
+          throw new IOException("Can't parse " + mapName + " list entry:" + line);
+        }
+        LOG.debug("add to " + mapName + "map:" + nameId[0] + " id:" + nameId[1]);
+        // HDFS can't differentiate duplicate names with simple authentication
+        Integer key = Integer.valueOf(nameId[1]);
+        String value = nameId[0];
+        if (map.containsKey(key)) {
+          LOG.error(String.format(
+              "Got duplicate id:(%d, %s), existing entry: (%d, %s).\n%s", key,
+              value, key, map.get(key), DUPLICATE_NAME_ID_DEBUG_INFO));
+          throw new DuplicateNameOrIdException("Got duplicate id.");
+        }
+        if (map.containsValue(nameId[0])) {
+          LOG.error(String.format(
+              "Got duplicate name:(%d, %s), existing entry: (%d, %s) \n%s",
+              key, value, map.inverse().get(value), value,
+              DUPLICATE_NAME_ID_DEBUG_INFO));
+          throw new DuplicateNameOrIdException("Got duplicate name");
         }
-        LOG.debug("add " + name + ":" + nameId[0] + " id:" + nameId[1]);
         map.put(Integer.valueOf(nameId[1]), nameId[0]);
       }
-      LOG.info("Updated " + name + " map size:" + map.size());
+      LOG.info("Updated " + mapName + " map size:" + map.size());
       
     } catch (IOException e) {
-      LOG.error("Can't update map " + name);
+      LOG.error("Can't update " + mapName + " map");
       throw e;
     } finally {
       if (br != null) {
@@ -101,24 +142,26 @@ public class IdUserGroup {
     }
   }
 
-  synchronized public void updateMaps() {
+  synchronized public void updateMaps() throws IOException {
     BiMap<Integer, String> uMap = HashBiMap.create();
     BiMap<Integer, String> gMap = HashBiMap.create();
 
-    try {
-      if (OS.startsWith("Linux")) {
-        updateMapInternal(uMap, "user", LINUX_GET_ALL_USERS_CMD, ":");
-        updateMapInternal(gMap, "group", LINUX_GET_ALL_GROUPS_CMD, ":");
-      } else if (OS.startsWith("Mac")) {
-        updateMapInternal(uMap, "user", MAC_GET_ALL_USERS_CMD, "\\s+");
-        updateMapInternal(gMap, "group", MAC_GET_ALL_GROUPS_CMD, "\\s+");
-      } else {
-        throw new IOException("Platform is not supported:" + OS);
-      }
-    } catch (IOException e) {
-      LOG.error("Can't update maps:" + e);
+    if (!OS.startsWith("Linux") && !OS.startsWith("Mac")) {
+      LOG.error("Platform is not supported:" + OS
+          + ". Can't update user map and group map and"
+          + " 'nobody' will be used for any user and group.");
       return;
     }
+
+    if (OS.startsWith("Linux")) {
+      updateMapInternal(uMap, "user", LINUX_GET_ALL_USERS_CMD, ":");
+      updateMapInternal(gMap, "group", LINUX_GET_ALL_GROUPS_CMD, ":");
+    } else {
+      // Mac
+      updateMapInternal(uMap, "user", MAC_GET_ALL_USERS_CMD, "\\s+");
+      updateMapInternal(gMap, "group", MAC_GET_ALL_GROUPS_CMD, "\\s+");
+    }
+
     uidNameMap = uMap;
     gidNameMap = gMap;
     lastUpdateTime = System.currentTimeMillis();

+ 56 - 0
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/nfs3/TestIdUserGroup.java

@@ -0,0 +1,56 @@
+/**
+ * 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.nfs.nfs3;
+
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
+import org.apache.hadoop.nfs.nfs3.IdUserGroup.DuplicateNameOrIdException;
+import org.junit.Test;
+
+import com.google.common.collect.BiMap;
+import com.google.common.collect.HashBiMap;
+
+public class TestIdUserGroup {
+
+  @Test
+  public void testDuplicates() throws IOException {
+    String GET_ALL_USERS_CMD = "echo \"root:x:0:0:root:/root:/bin/bash\n"
+        + "hdfs:x:11501:10787:Grid Distributed File System:/home/hdfs:/bin/bash\n"
+        + "hdfs:x:11502:10788:Grid Distributed File System:/home/hdfs:/bin/bash\""
+        + " | cut -d: -f1,3";
+    String GET_ALL_GROUPS_CMD = "echo \"hdfs:*:11501:hrt_hdfs\n"
+        + "mapred:x:497\n" + "mapred2:x:497\"" + " | cut -d: -f1,3";
+    // Maps for id to name map
+    BiMap<Integer, String> uMap = HashBiMap.create();
+    BiMap<Integer, String> gMap = HashBiMap.create();
+
+    try {
+      IdUserGroup.updateMapInternal(uMap, "user", GET_ALL_USERS_CMD, ":");
+      fail("didn't detect the duplicate name");
+    } catch (DuplicateNameOrIdException e) {
+    }
+    
+    try {
+      IdUserGroup.updateMapInternal(gMap, "group", GET_ALL_GROUPS_CMD, ":");
+      fail("didn't detect the duplicate id");
+    } catch (DuplicateNameOrIdException e) {
+    }
+  }
+}

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

@@ -228,6 +228,11 @@ Trunk (Unreleased)
 
     HDFS-5430. Support TTL on CacheDirectives. (wang)
 
+    HDFS-5536. Implement HTTP policy for Namenode and DataNode. (Haohui Mai via
+    jing9)
+
+    HDFS-5630. Hook up cache directive and pool usage statistics. (wang)
+
   OPTIMIZATIONS
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
 
@@ -421,6 +426,11 @@ Trunk (Unreleased)
     HDFS-5562. TestCacheDirectives and TestFsDatasetCache should stub out
     native mlock. (Colin McCabe and Akira Ajisaka via wang)
 
+    HDFS-5555. CacheAdmin commands fail when first listed NameNode is in
+    Standby (jxiang via cmccabe)
+
+    HDFS-5626. dfsadmin -report shows incorrect cache values. (cmccabe)
+
 Release 2.4.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -449,6 +459,8 @@ Release 2.4.0 - UNRELEASED
     HDFS-5444. Choose default web UI based on browser capabilities. (Haohui Mai
     via jing9)
 
+    HDFS-5514. FSNamesystem's fsLock should allow custom implementation (daryn)
+
   IMPROVEMENTS
 
     HDFS-5267. Remove volatile from LightWeightHashSet. (Junping Du via llu)
@@ -769,6 +781,9 @@ Release 2.3.0 - UNRELEASED
 
     HDFS-4997. libhdfs doesn't return correct error codes in most cases (cmccabe)
 
+    HDFS-5587. add debug information when NFS fails to start with duplicate user
+    or group names (brandonli)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

+ 4 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -109,8 +109,10 @@ import org.apache.hadoop.hdfs.client.ClientMmapManager;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveIterator;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolIterator;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
@@ -2324,12 +2326,7 @@ public class DFSClient implements java.io.Closeable {
   
   public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
       CacheDirectiveInfo filter) throws IOException {
-    checkOpen();
-    try {
-      return namenode.listCacheDirectives(0, filter);
-    } catch (RemoteException re) {
-      throw re.unwrapRemoteException();
-    }
+    return new CacheDirectiveIterator(namenode, filter);
   }
 
   public void addCachePool(CachePoolInfo info) throws IOException {
@@ -2360,12 +2357,7 @@ public class DFSClient implements java.io.Closeable {
   }
 
   public RemoteIterator<CachePoolEntry> listCachePools() throws IOException {
-    checkOpen();
-    try {
-      return namenode.listCachePools("");
-    } catch (RemoteException re) {
-      throw re.unwrapRemoteException();
-    }
+    return new CachePoolIterator(namenode);
   }
 
   /**

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
+import org.apache.hadoop.http.HttpConfig;
 
 /** 
  * This class contains constants for configuration keys used
@@ -358,6 +359,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final boolean DFS_SUPPORT_APPEND_DEFAULT = true;
   public static final String  DFS_HTTPS_ENABLE_KEY = "dfs.https.enable";
   public static final boolean DFS_HTTPS_ENABLE_DEFAULT = false;
+  public static final String  DFS_HTTP_POLICY_KEY = "dfs.http.policy";
+  public static final String  DFS_HTTP_POLICY_DEFAULT =  HttpConfig.Policy.HTTP_ONLY.name();
   public static final String  DFS_DEFAULT_CHUNK_VIEW_SIZE_KEY = "dfs.default.chunk.view.size";
   public static final int     DFS_DEFAULT_CHUNK_VIEW_SIZE_DEFAULT = 32*1024;
   public static final String  DFS_DATANODE_HTTPS_ADDRESS_KEY = "dfs.datanode.https.address";

+ 54 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hdfs;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY;
@@ -65,6 +67,7 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
@@ -78,6 +81,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.web.SWebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
@@ -1432,12 +1436,58 @@ public class DFSUtil {
         defaultKey : DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY;
   }
 
-  public static HttpServer.Builder loadSslConfToHttpServerBuilder(
-      HttpServer.Builder builder, Configuration sslConf) {
+  /**
+   * Get http policy. Http Policy is chosen as follows:
+   * <ol>
+   * <li>If hadoop.ssl.enabled is set, http endpoints are not started. Only
+   * https endpoints are started on configured https ports</li>
+   * <li>This configuration is overridden by dfs.https.enable configuration, if
+   * it is set to true. In that case, both http and https endpoints are stared.</li>
+   * <li>All the above configurations are overridden by dfs.http.policy
+   * configuration. With this configuration you can set http-only, https-only
+   * and http-and-https endpoints.</li>
+   * </ol>
+   * See hdfs-default.xml documentation for more details on each of the above
+   * configuration settings.
+   */
+  public static HttpConfig.Policy getHttpPolicy(Configuration conf) {
+    String httpPolicy = conf.get(DFSConfigKeys.DFS_HTTP_POLICY_KEY,
+        DFSConfigKeys.DFS_HTTP_POLICY_DEFAULT);
+
+    HttpConfig.Policy policy = HttpConfig.Policy.fromString(httpPolicy);
+
+    if (policy == HttpConfig.Policy.HTTP_ONLY) {
+      boolean httpsEnabled = conf.getBoolean(
+          DFSConfigKeys.DFS_HTTPS_ENABLE_KEY,
+          DFSConfigKeys.DFS_HTTPS_ENABLE_DEFAULT);
+
+      boolean hadoopSslEnabled = conf.getBoolean(
+          CommonConfigurationKeys.HADOOP_SSL_ENABLED_KEY,
+          CommonConfigurationKeys.HADOOP_SSL_ENABLED_DEFAULT);
+
+      if (hadoopSslEnabled) {
+        LOG.warn(CommonConfigurationKeys.HADOOP_SSL_ENABLED_KEY
+            + " is deprecated. Please use "
+            + DFSConfigKeys.DFS_HTTPS_ENABLE_KEY + ".");
+        policy = HttpConfig.Policy.HTTPS_ONLY;
+      } else if (httpsEnabled) {
+        LOG.warn(DFSConfigKeys.DFS_HTTPS_ENABLE_KEY
+            + " is deprecated. Please use "
+            + DFSConfigKeys.DFS_HTTPS_ENABLE_KEY + ".");
+        policy = HttpConfig.Policy.HTTP_AND_HTTPS;
+      }
+    }
+
+    conf.set(DFSConfigKeys.DFS_HTTP_POLICY_KEY, policy.name());
+    return policy;
+  }
+
+  public static HttpServer.Builder loadSslConfToHttpServerBuilder(HttpServer.Builder builder,
+      Configuration sslConf) {
     return builder
         .needsClientAuth(
-            sslConf.getBoolean(DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_KEY,
-                DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT))
+            sslConf.getBoolean(DFS_CLIENT_HTTPS_NEED_AUTH_KEY,
+                DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT))
         .keyPassword(sslConf.get("ssl.server.keystore.keypassword"))
         .keyStore(sslConf.get("ssl.server.keystore.location"),
             sslConf.get("ssl.server.keystore.password"),

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.hdfs;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configuration.DeprecationDelta;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 

+ 41 - 22
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirective.java

@@ -46,7 +46,9 @@ public final class CacheDirective implements IntrusiveCollection.Element {
 
   private long bytesNeeded;
   private long bytesCached;
-  private long filesAffected;
+  private long filesNeeded;
+  private long filesCached;
+
   private Element prev;
   private Element next;
 
@@ -58,9 +60,6 @@ public final class CacheDirective implements IntrusiveCollection.Element {
     Preconditions.checkArgument(replication > 0);
     this.replication = replication;
     this.expiryTime = expiryTime;
-    this.bytesNeeded = 0;
-    this.bytesCached = 0;
-    this.filesAffected = 0;
   }
 
   public long getId() {
@@ -112,7 +111,8 @@ public final class CacheDirective implements IntrusiveCollection.Element {
     return new CacheDirectiveStats.Builder().
         setBytesNeeded(bytesNeeded).
         setBytesCached(bytesCached).
-        setFilesAffected(filesAffected).
+        setFilesNeeded(filesNeeded).
+        setFilesCached(filesCached).
         setHasExpired(new Date().getTime() > expiryTime).
         build();
   }
@@ -131,7 +131,8 @@ public final class CacheDirective implements IntrusiveCollection.Element {
       append(", expiryTime: ").append(getExpiryTimeString()).
       append(", bytesNeeded:").append(bytesNeeded).
       append(", bytesCached:").append(bytesCached).
-      append(", filesAffected:").append(filesAffected).
+      append(", filesNeeded:").append(filesNeeded).
+      append(", filesCached:").append(filesCached).
       append(" }");
     return builder.toString();
   }
@@ -152,42 +153,60 @@ public final class CacheDirective implements IntrusiveCollection.Element {
     return new HashCodeBuilder().append(id).toHashCode();
   }
 
-  public long getBytesNeeded() {
-    return bytesNeeded;
+  //
+  // Stats related getters and setters
+  //
+
+  /**
+   * Resets the byte and file statistics being tracked by this CacheDirective.
+   */
+  public void resetStatistics() {
+    bytesNeeded = 0;
+    bytesCached = 0;
+    filesNeeded = 0;
+    filesCached = 0;
   }
 
-  public void clearBytesNeeded() {
-    this.bytesNeeded = 0;
+  public long getBytesNeeded() {
+    return bytesNeeded;
   }
 
-  public void addBytesNeeded(long toAdd) {
-    this.bytesNeeded += toAdd;
+  public void addBytesNeeded(long bytes) {
+    this.bytesNeeded += bytes;
+    pool.addBytesNeeded(bytes);
   }
 
   public long getBytesCached() {
     return bytesCached;
   }
 
-  public void clearBytesCached() {
-    this.bytesCached = 0;
+  public void addBytesCached(long bytes) {
+    this.bytesCached += bytes;
+    pool.addBytesCached(bytes);
   }
 
-  public void addBytesCached(long toAdd) {
-    this.bytesCached += toAdd;
+  public long getFilesNeeded() {
+    return filesNeeded;
   }
 
-  public long getFilesAffected() {
-    return filesAffected;
+  public void addFilesNeeded(long files) {
+    this.filesNeeded += files;
+    pool.addFilesNeeded(files);
   }
 
-  public void clearFilesAffected() {
-    this.filesAffected = 0;
+  public long getFilesCached() {
+    return filesCached;
   }
 
-  public void incrementFilesAffected() {
-    this.filesAffected++;
+  public void addFilesCached(long files) {
+    this.filesCached += files;
+    pool.addFilesCached(files);
   }
 
+  //
+  // IntrusiveCollection.Element implementation
+  //
+
   @SuppressWarnings("unchecked")
   @Override // IntrusiveCollection.Element
   public void insertInternal(IntrusiveCollection<? extends Element> list,

+ 56 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java

@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.BatchedRemoteIterator;
+
+/**
+ * CacheDirectiveIterator is a remote iterator that iterates cache directives.
+ * It supports retrying in case of namenode failover.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class CacheDirectiveIterator
+    extends BatchedRemoteIterator<Long, CacheDirectiveEntry> {
+
+  private final CacheDirectiveInfo filter;
+  private final ClientProtocol namenode;
+
+  public CacheDirectiveIterator(ClientProtocol namenode,
+      CacheDirectiveInfo filter) {
+    super(Long.valueOf(0));
+    this.namenode = namenode;
+    this.filter = filter;
+  }
+
+  @Override
+  public BatchedEntries<CacheDirectiveEntry> makeRequest(Long prevKey)
+      throws IOException {
+    return namenode.listCacheDirectives(prevKey, filter);
+  }
+
+  @Override
+  public Long elementToPrevKey(CacheDirectiveEntry entry) {
+    return entry.getInfo().getId();
+  }
+}

+ 35 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java

@@ -29,7 +29,8 @@ public class CacheDirectiveStats {
   public static class Builder {
     private long bytesNeeded;
     private long bytesCached;
-    private long filesAffected;
+    private long filesNeeded;
+    private long filesCached;
     private boolean hasExpired;
 
     /**
@@ -38,8 +39,8 @@ public class CacheDirectiveStats {
      * @return New CacheDirectiveStats.
      */
     public CacheDirectiveStats build() {
-      return new CacheDirectiveStats(bytesNeeded, bytesCached, filesAffected,
-          hasExpired);
+      return new CacheDirectiveStats(bytesNeeded, bytesCached, filesNeeded,
+          filesCached, hasExpired);
     }
 
     /**
@@ -71,13 +72,23 @@ public class CacheDirectiveStats {
     }
 
     /**
-     * Sets the files affected by this directive.
+     * Sets the files needed by this directive.
+     * @param filesNeeded The number of files needed
+     * @return This builder, for call chaining.
+     */
+    public Builder setFilesNeeded(long filesNeeded) {
+      this.filesNeeded = filesNeeded;
+      return this;
+    }
+
+    /**
+     * Sets the files cached by this directive.
      * 
-     * @param filesAffected The files affected.
+     * @param filesCached The number of files cached.
      * @return This builder, for call chaining.
      */
-    public Builder setFilesAffected(long filesAffected) {
-      this.filesAffected = filesAffected;
+    public Builder setFilesCached(long filesCached) {
+      this.filesCached = filesCached;
       return this;
     }
 
@@ -95,14 +106,16 @@ public class CacheDirectiveStats {
 
   private final long bytesNeeded;
   private final long bytesCached;
-  private final long filesAffected;
+  private final long filesNeeded;
+  private final long filesCached;
   private final boolean hasExpired;
 
   private CacheDirectiveStats(long bytesNeeded, long bytesCached,
-      long filesAffected, boolean hasExpired) {
+      long filesNeeded, long filesCached, boolean hasExpired) {
     this.bytesNeeded = bytesNeeded;
     this.bytesCached = bytesCached;
-    this.filesAffected = filesAffected;
+    this.filesNeeded = filesNeeded;
+    this.filesCached = filesCached;
     this.hasExpired = hasExpired;
   }
 
@@ -121,10 +134,17 @@ public class CacheDirectiveStats {
   }
 
   /**
-   * @return The files affected.
+   * @return The number of files needed.
+   */
+  public long getFilesNeeded() {
+    return filesNeeded;
+  }
+
+  /**
+   * @return The number of files cached.
    */
-  public long getFilesAffected() {
-    return filesAffected;
+  public long getFilesCached() {
+    return filesCached;
   }
 
   /**
@@ -140,7 +160,8 @@ public class CacheDirectiveStats {
     builder.append("{");
     builder.append("bytesNeeded: ").append(bytesNeeded);
     builder.append(", ").append("bytesCached: ").append(bytesCached);
-    builder.append(", ").append("filesAffected: ").append(filesAffected);
+    builder.append(", ").append("filesNeeded: ").append(filesNeeded);
+    builder.append(", ").append("filesCached: ").append(filesCached);
     builder.append(", ").append("hasExpired: ").append(hasExpired);
     builder.append("}");
     return builder.toString();

+ 53 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java

@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.BatchedRemoteIterator;
+
+/**
+ * CachePoolIterator is a remote iterator that iterates cache pools.
+ * It supports retrying in case of namenode failover.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class CachePoolIterator
+    extends BatchedRemoteIterator<String, CachePoolEntry> {
+
+  private final ClientProtocol namenode;
+
+  public CachePoolIterator(ClientProtocol namenode) {
+    super("");
+    this.namenode = namenode;
+  }
+
+  @Override
+  public BatchedEntries<CachePoolEntry> makeRequest(String prevKey)
+      throws IOException {
+    return namenode.listCachePools(prevKey);
+  }
+
+  @Override
+  public String elementToPrevKey(CachePoolEntry entry) {
+    return entry.getInfo().getPoolName();
+  }
+}

+ 26 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java

@@ -30,7 +30,8 @@ public class CachePoolStats {
   public static class Builder {
     private long bytesNeeded;
     private long bytesCached;
-    private long filesAffected;
+    private long filesNeeded;
+    private long filesCached;
 
     public Builder() {
     }
@@ -45,24 +46,33 @@ public class CachePoolStats {
       return this;
     }
 
-    public Builder setFilesAffected(long filesAffected) {
-      this.filesAffected = filesAffected;
+    public Builder setFilesNeeded(long filesNeeded) {
+      this.filesNeeded = filesNeeded;
+      return this;
+    }
+
+    public Builder setFilesCached(long filesCached) {
+      this.filesCached = filesCached;
       return this;
     }
 
     public CachePoolStats build() {
-      return new CachePoolStats(bytesNeeded, bytesCached, filesAffected);
+      return new CachePoolStats(bytesNeeded, bytesCached, filesNeeded,
+          filesCached);
     }
   };
 
   private final long bytesNeeded;
   private final long bytesCached;
-  private final long filesAffected;
+  private final long filesNeeded;
+  private final long filesCached;
 
-  private CachePoolStats(long bytesNeeded, long bytesCached, long filesAffected) {
+  private CachePoolStats(long bytesNeeded, long bytesCached, long filesNeeded,
+      long filesCached) {
     this.bytesNeeded = bytesNeeded;
     this.bytesCached = bytesCached;
-    this.filesAffected = filesAffected;
+    this.filesNeeded = filesNeeded;
+    this.filesCached = filesCached;
   }
 
   public long getBytesNeeded() {
@@ -70,18 +80,23 @@ public class CachePoolStats {
   }
 
   public long getBytesCached() {
-    return bytesNeeded;
+    return bytesCached;
+  }
+
+  public long getFilesNeeded() {
+    return filesNeeded;
   }
 
-  public long getFilesAffected() {
-    return filesAffected;
+  public long getFilesCached() {
+    return filesCached;
   }
 
   public String toString() {
     return new StringBuilder().append("{").
       append("bytesNeeded:").append(bytesNeeded).
       append(", bytesCached:").append(bytesCached).
-      append(", filesAffected:").append(filesAffected).
+      append(", filesNeeded:").append(filesNeeded).
+      append(", filesCached:").append(filesCached).
       append("}").toString();
   }
 }

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -28,9 +28,9 @@ import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -1135,10 +1135,10 @@ public interface ClientProtocol {
    *               listCacheDirectives.
    * @param filter Parameters to use to filter the list results, 
    *               or null to display all directives visible to us.
-   * @return A RemoteIterator which returns CacheDirectiveInfo objects.
+   * @return A batch of CacheDirectiveEntry objects.
    */
   @Idempotent
-  public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
+  public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(
       long prevId, CacheDirectiveInfo filter) throws IOException;
 
   /**
@@ -1176,9 +1176,9 @@ public interface ClientProtocol {
    * 
    * @param prevPool name of the last pool listed, or the empty string if this is
    *          the first invocation of listCachePools
-   * @return A RemoteIterator which returns CachePool objects.
+   * @return A batch of CachePoolEntry objects.
    */
   @Idempotent
-  public RemoteIterator<CachePoolEntry> listCachePools(String prevPool)
+  public BatchedEntries<CachePoolEntry> listCachePools(String prevPool)
       throws IOException;
 }

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java

@@ -315,9 +315,9 @@ public class DatanodeInfo extends DatanodeID implements Node {
     buffer.append("DFS Remaining: " +r+ " ("+StringUtils.byteDesc(r)+")"+"\n");
     buffer.append("DFS Used%: "+percent2String(usedPercent) + "\n");
     buffer.append("DFS Remaining%: "+percent2String(remainingPercent) + "\n");
-    buffer.append("Configured Cache Capacity: "+c+" ("+StringUtils.byteDesc(cc)+")"+"\n");
-    buffer.append("Cache Used: "+cu+" ("+StringUtils.byteDesc(u)+")"+"\n");
-    buffer.append("Cache Remaining: " +cr+ " ("+StringUtils.byteDesc(r)+")"+"\n");
+    buffer.append("Configured Cache Capacity: "+cc+" ("+StringUtils.byteDesc(cc)+")"+"\n");
+    buffer.append("Cache Used: "+cu+" ("+StringUtils.byteDesc(cu)+")"+"\n");
+    buffer.append("Cache Remaining: " +cr+ " ("+StringUtils.byteDesc(cr)+")"+"\n");
     buffer.append("Cache Used%: "+percent2String(cacheUsedPercent) + "\n");
     buffer.append("Cache Remaining%: "+percent2String(cacheRemainingPercent) + "\n");
 

+ 10 - 35
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -24,12 +24,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
 import org.apache.hadoop.fs.Options.Rename;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
-import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
@@ -52,8 +49,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowS
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
@@ -109,7 +104,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCa
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
@@ -176,9 +170,7 @@ import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRespons
 import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto;
 import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenResponseProto;
 import org.apache.hadoop.security.token.Token;
-import org.apache.commons.lang.StringUtils;
 
-import com.google.common.primitives.Shorts;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 
@@ -1084,21 +1076,13 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     try {
       CacheDirectiveInfo filter =
           PBHelper.convert(request.getFilter());
-      RemoteIterator<CacheDirectiveEntry> iter =
-         server.listCacheDirectives(request.getPrevId(), filter);
+      BatchedEntries<CacheDirectiveEntry> entries =
+        server.listCacheDirectives(request.getPrevId(), filter);
       ListCacheDirectivesResponseProto.Builder builder =
           ListCacheDirectivesResponseProto.newBuilder();
-      long prevId = 0;
-      while (iter.hasNext()) {
-        CacheDirectiveEntry entry = iter.next();
-        builder.addElements(PBHelper.convert(entry));
-        prevId = entry.getInfo().getId();
-      }
-      if (prevId == 0) {
-        builder.setHasMore(false);
-      } else {
-        iter = server.listCacheDirectives(prevId, filter);
-        builder.setHasMore(iter.hasNext());
+      builder.setHasMore(entries.hasMore());
+      for (int i=0, n=entries.size(); i<n; i++) {
+        builder.addElements(PBHelper.convert(entries.get(i)));
       }
       return builder.build();
     } catch (IOException e) {
@@ -1143,22 +1127,13 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public ListCachePoolsResponseProto listCachePools(RpcController controller,
       ListCachePoolsRequestProto request) throws ServiceException {
     try {
-      RemoteIterator<CachePoolEntry> iter =
+      BatchedEntries<CachePoolEntry> entries =
         server.listCachePools(request.getPrevPoolName());
       ListCachePoolsResponseProto.Builder responseBuilder =
         ListCachePoolsResponseProto.newBuilder();
-      String prevPoolName = null;
-      while (iter.hasNext()) {
-        CachePoolEntry entry = iter.next();
-        responseBuilder.addEntries(PBHelper.convert(entry));
-        prevPoolName = entry.getInfo().getPoolName();
-      }
-      // fill in hasNext
-      if (prevPoolName == null) {
-        responseBuilder.setHasMore(false);
-      } else {
-        iter = server.listCachePools(prevPoolName);
-        responseBuilder.setHasMore(iter.hasNext());
+      responseBuilder.setHasMore(entries.hasMore());
+      for (int i=0, n=entries.size(); i<n; i++) {
+        responseBuilder.addEntries(PBHelper.convert(entries.get(i)));
       }
       return responseBuilder.build();
     } catch (IOException e) {

+ 20 - 64
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -24,7 +24,6 @@ import java.util.Arrays;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.BatchedRemoteIterator;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
@@ -32,7 +31,6 @@ import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
@@ -1065,46 +1063,23 @@ public class ClientNamenodeProtocolTranslatorPB implements
     }
   }
 
-  private class CacheEntriesIterator
-    extends BatchedRemoteIterator<Long, CacheDirectiveEntry> {
-      private final CacheDirectiveInfo filter;
-
-    public CacheEntriesIterator(long prevKey,
-        CacheDirectiveInfo filter) {
-      super(prevKey);
-      this.filter = filter;
-    }
-
-    @Override
-    public BatchedEntries<CacheDirectiveEntry> makeRequest(
-        Long nextKey) throws IOException {
-      ListCacheDirectivesResponseProto response;
-      try {
-        response = rpcProxy.listCacheDirectives(null,
-            ListCacheDirectivesRequestProto.newBuilder().
-                setPrevId(nextKey).
-                setFilter(PBHelper.convert(filter)).
-                build());
-      } catch (ServiceException e) {
-        throw ProtobufHelper.getRemoteException(e);
-      }
-      return new BatchedCacheEntries(response);
-    }
-
-    @Override
-    public Long elementToPrevKey(CacheDirectiveEntry element) {
-      return element.getInfo().getId();
-    }
-  }
-
   @Override
-  public RemoteIterator<CacheDirectiveEntry>
+  public BatchedEntries<CacheDirectiveEntry>
       listCacheDirectives(long prevId,
           CacheDirectiveInfo filter) throws IOException {
     if (filter == null) {
       filter = new CacheDirectiveInfo.Builder().build();
     }
-    return new CacheEntriesIterator(prevId, filter);
+    try {
+      return new BatchedCacheEntries(
+        rpcProxy.listCacheDirectives(null,
+          ListCacheDirectivesRequestProto.newBuilder().
+            setPrevId(prevId).
+            setFilter(PBHelper.convert(filter)).
+            build()));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
   }
 
   @Override
@@ -1167,35 +1142,16 @@ public class ClientNamenodeProtocolTranslatorPB implements
     }
   }
 
-  private class CachePoolIterator 
-      extends BatchedRemoteIterator<String, CachePoolEntry> {
-
-    public CachePoolIterator(String prevKey) {
-      super(prevKey);
-    }
-
-    @Override
-    public BatchedEntries<CachePoolEntry> makeRequest(String prevKey)
-        throws IOException {
-      try {
-        return new BatchedCachePoolEntries(
-            rpcProxy.listCachePools(null, 
-              ListCachePoolsRequestProto.newBuilder().
-                setPrevPoolName(prevKey).build()));
-      } catch (ServiceException e) {
-        throw ProtobufHelper.getRemoteException(e);
-      }
-    }
-
-    @Override
-    public String elementToPrevKey(CachePoolEntry entry) {
-      return entry.getInfo().getPoolName();
-    }
-  }
-
   @Override
-  public RemoteIterator<CachePoolEntry> listCachePools(String prevKey)
+  public BatchedEntries<CachePoolEntry> listCachePools(String prevKey)
       throws IOException {
-    return new CachePoolIterator(prevKey);
+    try {
+      return new BatchedCachePoolEntries(
+        rpcProxy.listCachePools(null,
+          ListCachePoolsRequestProto.newBuilder().
+            setPrevPoolName(prevKey).build()));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
   }
 }

+ 8 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -1753,7 +1753,8 @@ public class PBHelper {
         CacheDirectiveStatsProto.newBuilder();
     builder.setBytesNeeded(stats.getBytesNeeded());
     builder.setBytesCached(stats.getBytesCached());
-    builder.setFilesAffected(stats.getFilesAffected());
+    builder.setFilesNeeded(stats.getFilesNeeded());
+    builder.setFilesCached(stats.getFilesCached());
     builder.setHasExpired(stats.hasExpired());
     return builder.build();
   }
@@ -1762,7 +1763,8 @@ public class PBHelper {
     CacheDirectiveStats.Builder builder = new CacheDirectiveStats.Builder();
     builder.setBytesNeeded(proto.getBytesNeeded());
     builder.setBytesCached(proto.getBytesCached());
-    builder.setFilesAffected(proto.getFilesAffected());
+    builder.setFilesNeeded(proto.getFilesNeeded());
+    builder.setFilesCached(proto.getFilesCached());
     builder.setHasExpired(proto.getHasExpired());
     return builder.build();
   }
@@ -1822,7 +1824,8 @@ public class PBHelper {
     CachePoolStatsProto.Builder builder = CachePoolStatsProto.newBuilder();
     builder.setBytesNeeded(stats.getBytesNeeded());
     builder.setBytesCached(stats.getBytesCached());
-    builder.setFilesAffected(stats.getFilesAffected());
+    builder.setFilesNeeded(stats.getFilesNeeded());
+    builder.setFilesCached(stats.getFilesCached());
     return builder.build();
   }
 
@@ -1830,7 +1833,8 @@ public class PBHelper {
     CachePoolStats.Builder builder = new CachePoolStats.Builder();
     builder.setBytesNeeded(proto.getBytesNeeded());
     builder.setBytesCached(proto.getBytesCached());
-    builder.setFilesAffected(proto.getFilesAffected());
+    builder.setFilesNeeded(proto.getFilesNeeded());
+    builder.setFilesCached(proto.getFilesCached());
     return builder.build();
   }
 

+ 44 - 23
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.protocol.CacheDirective;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.CacheManager;
+import org.apache.hadoop.hdfs.server.namenode.CachePool;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -198,6 +199,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
     scannedBlocks = 0;
     namesystem.writeLock();
     try {
+      resetStatistics();
       rescanCacheDirectives();
       rescanCachedBlockMap();
       blockManager.getDatanodeManager().resetLastCachingDirectiveSentTime();
@@ -206,6 +208,15 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
     }
   }
 
+  private void resetStatistics() {
+    for (CachePool pool: cacheManager.getCachePools()) {
+      pool.resetStatistics();
+    }
+    for (CacheDirective directive: cacheManager.getCacheDirectives()) {
+      directive.resetStatistics();
+    }
+  }
+
   /**
    * Scan all CacheDirectives.  Use the information to figure out
    * what cache replication factor each block should have.
@@ -213,11 +224,9 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
   private void rescanCacheDirectives() {
     FSDirectory fsDir = namesystem.getFSDirectory();
     final long now = new Date().getTime();
-    for (CacheDirective directive : cacheManager.getEntriesById().values()) {
-      // Reset the directive
-      directive.clearBytesNeeded();
-      directive.clearBytesCached();
-      directive.clearFilesAffected();
+    for (CacheDirective directive : cacheManager.getCacheDirectives()) {
+      // Reset the directive's statistics
+      directive.resetStatistics();
       // Skip processing this entry if it has expired
       LOG.info("Directive expiry is at " + directive.getExpiryTime());
       if (directive.getExpiryTime() > 0 && directive.getExpiryTime() <= now) {
@@ -262,26 +271,34 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
   
   /**
    * Apply a CacheDirective to a file.
-   *
-   * @param pce       The CacheDirective to apply.
-   * @param file      The file.
+   * 
+   * @param directive The CacheDirective to apply.
+   * @param file The file.
    */
-  private void rescanFile(CacheDirective pce, INodeFile file) {
-    pce.incrementFilesAffected();
+  private void rescanFile(CacheDirective directive, INodeFile file) {
     BlockInfo[] blockInfos = file.getBlocks();
-    long cachedTotal = 0;
+
+    // Increment the "needed" statistics
+    directive.addFilesNeeded(1);
     long neededTotal = 0;
+    for (BlockInfo blockInfo : blockInfos) {
+      long neededByBlock = 
+          directive.getReplication() * blockInfo.getNumBytes();
+       neededTotal += neededByBlock;
+    }
+    directive.addBytesNeeded(neededTotal);
+
+    // TODO: Enforce per-pool quotas
+
+    long cachedTotal = 0;
     for (BlockInfo blockInfo : blockInfos) {
       if (!blockInfo.getBlockUCState().equals(BlockUCState.COMPLETE)) {
         // We don't try to cache blocks that are under construction.
         continue;
       }
-      long neededByBlock = 
-         pce.getReplication() * blockInfo.getNumBytes();
-      neededTotal += neededByBlock;
       Block block = new Block(blockInfo.getBlockId());
       CachedBlock ncblock = new CachedBlock(block.getBlockId(),
-          pce.getReplication(), mark);
+          directive.getReplication(), mark);
       CachedBlock ocblock = cachedBlocks.get(ncblock);
       if (ocblock == null) {
         cachedBlocks.put(ncblock);
@@ -294,26 +311,30 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
         // both get them added to their bytesCached.
         List<DatanodeDescriptor> cachedOn =
             ocblock.getDatanodes(Type.CACHED);
-        long cachedByBlock = Math.min(cachedOn.size(), pce.getReplication()) *
-            blockInfo.getNumBytes();
+        long cachedByBlock = Math.min(cachedOn.size(),
+            directive.getReplication()) * blockInfo.getNumBytes();
         cachedTotal += cachedByBlock;
 
         if (mark != ocblock.getMark()) {
           // Mark hasn't been set in this scan, so update replication and mark.
-          ocblock.setReplicationAndMark(pce.getReplication(), mark);
+          ocblock.setReplicationAndMark(directive.getReplication(), mark);
         } else {
           // Mark already set in this scan.  Set replication to highest value in
           // any CacheDirective that covers this file.
           ocblock.setReplicationAndMark((short)Math.max(
-              pce.getReplication(), ocblock.getReplication()), mark);
+              directive.getReplication(), ocblock.getReplication()), mark);
         }
       }
     }
-    pce.addBytesNeeded(neededTotal);
-    pce.addBytesCached(cachedTotal);
+    // Increment the "cached" statistics
+    directive.addBytesCached(cachedTotal);
+    if (cachedTotal == neededTotal) {
+      directive.addFilesCached(1);
+    }
     if (LOG.isTraceEnabled()) {
-      LOG.debug("Directive " + pce.getId() + " is caching " +
-          file.getFullPathName() + ": " + cachedTotal + "/" + neededTotal);
+      LOG.trace("Directive " + directive.getId() + " is caching " +
+          file.getFullPathName() + ": " + cachedTotal + "/" + neededTotal +
+          " bytes");
     }
   }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java

@@ -526,7 +526,7 @@ class BPServiceActor implements Runnable {
       long createCost = createTime - startTime;
       long sendCost = sendTime - createTime;
       dn.getMetrics().addCacheReport(sendCost);
-      LOG.info("CacheReport of " + blockIds.size()
+      LOG.debug("CacheReport of " + blockIds.size()
           + " block(s) took " + createCost + " msec to generate and "
           + sendCost + " msecs for RPC and NN processing");
     }

+ 49 - 22
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-
 import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
 import static org.apache.hadoop.util.ExitUtil.terminate;
 
@@ -120,6 +119,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.resources.Param;
+import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.ReadaheadPool;
@@ -234,9 +234,11 @@ public class DataNode extends Configured
   private DNConf dnConf;
   private volatile boolean heartbeatsDisabledForTests = false;
   private DataStorage storage = null;
+
   private HttpServer infoServer = null;
   private int infoPort;
   private int infoSecurePort;
+
   DataNodeMetrics metrics;
   private InetSocketAddress streamingAddr;
   
@@ -332,7 +334,7 @@ public class DataNode extends Configured
    * explicitly configured in the given config, then it is determined
    * via the DNS class.
    *
-   * @param config
+   * @param config configuration
    * @return the hostname (NB: may not be a FQDN)
    * @throws UnknownHostException if the dfs.datanode.dns.interface
    *    option is used and the hostname can not be determined
@@ -350,39 +352,54 @@ public class DataNode extends Configured
     return name;
   }
 
+  /**
+   * @see DFSUtil#getHttpPolicy(org.apache.hadoop.conf.Configuration)
+   * for information related to the different configuration options and
+   * Http Policy is decided.
+   */
   private void startInfoServer(Configuration conf) throws IOException {
-    // create a servlet to serve full-file content
+    HttpServer.Builder builder = new HttpServer.Builder().setName("datanode")
+        .setConf(conf).setACL(new AccessControlList(conf.get(DFS_ADMIN, " ")));
+
+    HttpConfig.Policy policy = DFSUtil.getHttpPolicy(conf);
     InetSocketAddress infoSocAddr = DataNode.getInfoAddr(conf);
     String infoHost = infoSocAddr.getHostName();
-    int tmpInfoPort = infoSocAddr.getPort();
-    HttpServer.Builder builder = new HttpServer.Builder().setName("datanode")
-        .addEndpoint(URI.create("http://" + NetUtils.getHostPortString(infoSocAddr)))
-        .setFindPort(tmpInfoPort == 0).setConf(conf)
-        .setACL(new AccessControlList(conf.get(DFS_ADMIN, " ")));
 
-    LOG.info("Opened info server at " + infoHost + ":" + tmpInfoPort);
-    if (conf.getBoolean(DFS_HTTPS_ENABLE_KEY, false)) {
+    if (policy.isHttpEnabled()) {
+      if (secureResources == null) {
+        int port = infoSocAddr.getPort();
+        builder.addEndpoint(URI.create("http://" + infoHost + ":" + port));
+        if (port == 0) {
+          builder.setFindPort(true);
+        }
+      } else {
+        // The http socket is created externally using JSVC, we add it in
+        // directly.
+        builder.setConnector(secureResources.getListener());
+      }
+    }
+
+    if (policy.isHttpsEnabled()) {
       InetSocketAddress secInfoSocAddr = NetUtils.createSocketAddr(conf.get(
           DFS_DATANODE_HTTPS_ADDRESS_KEY, infoHost + ":" + 0));
-      builder.addEndpoint(URI.create("https://"
-          + NetUtils.getHostPortString(secInfoSocAddr)));
+
       Configuration sslConf = new Configuration(false);
-      sslConf.setBoolean(DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_KEY, conf
-          .getBoolean(DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_KEY,
-              DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT));
       sslConf.addResource(conf.get(
           DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY,
           DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_DEFAULT));
+      sslConf.setBoolean(DFS_CLIENT_HTTPS_NEED_AUTH_KEY, conf.getBoolean(
+          DFS_CLIENT_HTTPS_NEED_AUTH_KEY, DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT));
       DFSUtil.loadSslConfToHttpServerBuilder(builder, sslConf);
 
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Datanode listening for SSL on " + secInfoSocAddr);
+      int port = secInfoSocAddr.getPort();
+      if (port == 0) {
+        builder.setFindPort(true);
       }
-      infoSecurePort = secInfoSocAddr.getPort();
+      builder.addEndpoint(URI.create("https://" + infoHost + ":" + port));
     }
 
-    this.infoServer = (secureResources == null) ? builder.build() :
-      builder.setConnector(secureResources.getListener()).build();
+    this.infoServer = builder.build();
+
     this.infoServer.addInternalServlet(null, "/streamFile/*", StreamFile.class);
     this.infoServer.addInternalServlet(null, "/getFileChecksum/*",
         FileChecksumServlets.GetServlet.class);
@@ -398,9 +415,17 @@ public class DataNode extends Configured
           WebHdfsFileSystem.PATH_PREFIX + "/*");
     }
     this.infoServer.start();
-    this.infoPort = infoServer.getConnectorAddress(0).getPort();
+
+    int connIdx = 0;
+    if (policy.isHttpEnabled()) {
+      infoPort = infoServer.getConnectorAddress(connIdx++).getPort();
+    }
+
+    if (policy.isHttpsEnabled()) {
+      infoSecurePort = infoServer.getConnectorAddress(connIdx).getPort();
+    }
   }
-  
+
   private void startPlugins(Configuration conf) {
     plugins = conf.getInstances(DFS_DATANODE_PLUGINS_KEY, ServicePlugin.class);
     for (ServicePlugin p: plugins) {
@@ -738,6 +763,8 @@ public class DataNode extends Configured
           ulimit));
       }
     }
+    LOG.info("Starting DataNode with maxLockedMemory = " +
+        dnConf.maxLockedMemory);
 
     storage = new DataStorage();
     

+ 44 - 57
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java

@@ -16,27 +16,20 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.nio.channels.ServerSocketChannel;
-import java.security.GeneralSecurityException;
 
 import org.apache.commons.daemon.Daemon;
 import org.apache.commons.daemon.DaemonContext;
 import org.apache.hadoop.conf.Configuration;
-
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.ssl.SSLFactory;
 import org.mortbay.jetty.Connector;
-import org.mortbay.jetty.nio.SelectChannelConnector;
-import org.mortbay.jetty.security.SslSocketConnector;
-
-import javax.net.ssl.SSLServerSocketFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -65,7 +58,6 @@ public class SecureDataNodeStarter implements Daemon {
   
   private String [] args;
   private SecureResources resources;
-  private SSLFactory sslFactory;
 
   @Override
   public void init(DaemonContext context) throws Exception {
@@ -74,9 +66,7 @@ public class SecureDataNodeStarter implements Daemon {
     
     // Stash command-line arguments for regular datanode
     args = context.getArguments();
-    
-    sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
-    resources = getSecureResources(sslFactory, conf);
+    resources = getSecureResources(conf);
   }
 
   @Override
@@ -84,68 +74,65 @@ public class SecureDataNodeStarter implements Daemon {
     System.err.println("Starting regular datanode initialization");
     DataNode.secureMain(args, resources);
   }
-  
-  @Override public void destroy() {
-    sslFactory.destroy();
-  }
 
+  @Override public void destroy() {}
   @Override public void stop() throws Exception { /* Nothing to do */ }
 
+  /**
+   * Acquire privileged resources (i.e., the privileged ports) for the data
+   * node. The privileged resources consist of the port of the RPC server and
+   * the port of HTTP (not HTTPS) server.
+   */
   @VisibleForTesting
-  public static SecureResources getSecureResources(final SSLFactory sslFactory,
-                                  Configuration conf) throws Exception {
+  public static SecureResources getSecureResources(Configuration conf)
+      throws Exception {
+    HttpConfig.Policy policy = DFSUtil.getHttpPolicy(conf);
+
     // Obtain secure port for data streaming to datanode
     InetSocketAddress streamingAddr  = DataNode.getStreamingAddr(conf);
-    int socketWriteTimeout = conf.getInt(DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
+    int socketWriteTimeout = conf.getInt(
+        DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
         HdfsServerConstants.WRITE_TIMEOUT);
-    
+
     ServerSocket ss = (socketWriteTimeout > 0) ? 
         ServerSocketChannel.open().socket() : new ServerSocket();
     ss.bind(streamingAddr, 0);
-    
+
     // Check that we got the port we need
     if (ss.getLocalPort() != streamingAddr.getPort()) {
-      throw new RuntimeException("Unable to bind on specified streaming port in secure " +
-          "context. Needed " + streamingAddr.getPort() + ", got " + ss.getLocalPort());
+      throw new RuntimeException(
+          "Unable to bind on specified streaming port in secure "
+              + "context. Needed " + streamingAddr.getPort() + ", got "
+              + ss.getLocalPort());
     }
 
-    // Obtain secure listener for web server
-    Connector listener;
-    if (HttpConfig.isSecure()) {
-      try {
-        sslFactory.init();
-      } catch (GeneralSecurityException ex) {
-        throw new IOException(ex);
-      }
-      SslSocketConnector sslListener = new SslSocketConnector() {
-        @Override
-        protected SSLServerSocketFactory createFactory() throws Exception {
-          return sslFactory.createSSLServerSocketFactory();
-        }
-      };
-      listener = sslListener;
-    } else {
+    System.err.println("Opened streaming server at " + streamingAddr);
+
+    // Bind a port for the web server. The code intends to bind HTTP server to
+    // privileged port only, as the client can authenticate the server using
+    // certificates if they are communicating through SSL.
+    Connector listener = null;
+    if (policy.isHttpEnabled()) {
       listener = HttpServer.createDefaultChannelConnector();
-    }
+      InetSocketAddress infoSocAddr = DataNode.getInfoAddr(conf);
+      listener.setHost(infoSocAddr.getHostName());
+      listener.setPort(infoSocAddr.getPort());
+      // Open listener here in order to bind to port as root
+      listener.open();
+      if (listener.getPort() != infoSocAddr.getPort()) {
+        throw new RuntimeException("Unable to bind on specified info port in secure " +
+            "context. Needed " + streamingAddr.getPort() + ", got " + ss.getLocalPort());
+      }
+      System.err.println("Successfully obtained privileged resources (streaming port = "
+          + ss + " ) (http listener port = " + listener.getConnection() +")");
 
-    InetSocketAddress infoSocAddr = DataNode.getInfoAddr(conf);
-    listener.setHost(infoSocAddr.getHostName());
-    listener.setPort(infoSocAddr.getPort());
-    // Open listener here in order to bind to port as root
-    listener.open();
-    if (listener.getPort() != infoSocAddr.getPort()) {
-      throw new RuntimeException("Unable to bind on specified info port in secure " +
-          "context. Needed " + streamingAddr.getPort() + ", got " + ss.getLocalPort());
-    }
-    System.err.println("Successfully obtained privileged resources (streaming port = "
-        + ss + " ) (http listener port = " + listener.getConnection() +")");
-    
-    if ((ss.getLocalPort() > 1023 || listener.getPort() > 1023) &&
-        UserGroupInformation.isSecurityEnabled()) {
-      throw new RuntimeException("Cannot start secure datanode with unprivileged ports");
+      if ((ss.getLocalPort() > 1023 || listener.getPort() > 1023) &&
+          UserGroupInformation.isSecurityEnabled()) {
+        throw new RuntimeException("Cannot start secure datanode with unprivileged ports");
+      }
+      System.err.println("Opened info server at " + infoSocAddr);
     }
-    System.err.println("Opened streaming server at " + streamingAddr);
-    System.err.println("Opened info server at " + infoSocAddr);
+
     return new SecureResources(ss, listener);
   }
 

+ 4 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java

@@ -123,11 +123,6 @@ public class BackupNode extends NameNode {
     String addr = conf.get(BN_HTTP_ADDRESS_NAME_KEY, BN_HTTP_ADDRESS_DEFAULT);
     return NetUtils.createSocketAddr(addr);
   }
-  
-  @Override // NameNode
-  protected void setHttpServerAddress(Configuration conf){
-    conf.set(BN_HTTP_ADDRESS_NAME_KEY, NetUtils.getHostPortString(getHttpAddress()));
-  }
 
   @Override // NameNode
   protected void loadNamesystem(Configuration conf) throws IOException {
@@ -164,6 +159,10 @@ public class BackupNode extends NameNode {
     registerWith(nsInfo);
     // Checkpoint daemon should start after the rpc server started
     runCheckpointDaemon(conf);
+    InetSocketAddress addr = getHttpAddress();
+    if (addr != null) {
+      conf.set(BN_HTTP_ADDRESS_NAME_KEY, NetUtils.getHostPortString(getHttpAddress()));
+    }
   }
 
   @Override

+ 14 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java

@@ -31,6 +31,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
@@ -238,9 +239,20 @@ public final class CacheManager {
     return active;
   }
 
-  public TreeMap<Long, CacheDirective> getEntriesById() {
+  /**
+   * @return Unmodifiable view of the collection of CachePools.
+   */
+  public Collection<CachePool> getCachePools() {
+    assert namesystem.hasReadLock();
+    return Collections.unmodifiableCollection(cachePools.values());
+  }
+
+  /**
+   * @return Unmodifiable view of the collection of CacheDirectives.
+   */
+  public Collection<CacheDirective> getCacheDirectives() {
     assert namesystem.hasReadLock();
-    return directivesById;
+    return Collections.unmodifiableCollection(directivesById.values());
   }
   
   @VisibleForTesting

+ 52 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java

@@ -70,9 +70,14 @@ public final class CachePool {
    */
   @Nonnull
   private FsPermission mode;
-  
+
   private int weight;
 
+  private long bytesNeeded;
+  private long bytesCached;
+  private long filesNeeded;
+  private long filesCached;
+
   public final static class DirectiveList
       extends IntrusiveCollection<CacheDirective> {
     private CachePool cachePool;
@@ -202,6 +207,48 @@ public final class CachePool {
         setWeight(weight);
   }
 
+  /**
+   * Resets statistics related to this CachePool
+   */
+  public void resetStatistics() {
+    bytesNeeded = 0;
+    bytesCached = 0;
+    filesNeeded = 0;
+    filesCached = 0;
+  }
+
+  public void addBytesNeeded(long bytes) {
+    bytesNeeded += bytes;
+  }
+
+  public void addBytesCached(long bytes) {
+    bytesCached += bytes;
+  }
+
+  public void addFilesNeeded(long files) {
+    filesNeeded += files;
+  }
+
+  public void addFilesCached(long files) {
+    filesCached += files;
+  }
+
+  public long getBytesNeeded() {
+    return bytesNeeded;
+  }
+
+  public long getBytesCached() {
+    return bytesCached;
+  }
+
+  public long getFilesNeeded() {
+    return filesNeeded;
+  }
+
+  public long getFilesCached() {
+    return filesCached;
+  }
+
   /**
    * Get statistics about this CachePool.
    *
@@ -209,9 +256,10 @@ public final class CachePool {
    */
   private CachePoolStats getStats() {
     return new CachePoolStats.Builder().
-        setBytesNeeded(0).
-        setBytesCached(0).
-        setFilesAffected(0).
+        setBytesNeeded(bytesNeeded).
+        setBytesCached(bytesCached).
+        setFilesNeeded(filesNeeded).
+        setFilesCached(filesCached).
         build();
   }
 

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

@@ -117,7 +117,7 @@ public class EditLogFileInputStream extends EditLogInputStream {
    */
   public static EditLogInputStream fromUrl(
       URLConnectionFactory connectionFactory, URL url, long startTxId,
- long endTxId, boolean inProgress) {
+      long endTxId, boolean inProgress) {
     return new EditLogFileInputStream(new URLLog(connectionFactory, url),
         startTxId, endTxId, inProgress);
   }

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

@@ -462,7 +462,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private final long accessTimePrecision;
 
   /** Lock to protect FSNamesystem. */
-  private ReentrantReadWriteLock fsLock;
+  private FSNamesystemLock fsLock;
 
   /**
    * Used when this NN is in standby state to read from the shared edit log.
@@ -645,7 +645,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       throws IOException {
     boolean fair = conf.getBoolean("dfs.namenode.fslock.fair", true);
     LOG.info("fsLock is fair:" + fair);
-    fsLock = new ReentrantReadWriteLock(fair);
+    fsLock = new FSNamesystemLock(fair);
     try {
       resourceRecheckInterval = conf.getLong(
           DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY,
@@ -6763,12 +6763,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   
   @VisibleForTesting
   void setFsLockForTests(ReentrantReadWriteLock lock) {
-    this.fsLock = lock;
+    this.fsLock.coarseLock = lock;
   }
   
   @VisibleForTesting
   ReentrantReadWriteLock getFsLockForTests() {
-    return fsLock;
+    return fsLock.coarseLock;
   }
 
   @VisibleForTesting

+ 61 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java

@@ -0,0 +1,61 @@
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Mimics a ReentrantReadWriteLock so more sophisticated locking capabilities
+ * are possible.
+ */
+class FSNamesystemLock implements ReadWriteLock {
+  @VisibleForTesting
+  protected ReentrantReadWriteLock coarseLock;
+  
+  FSNamesystemLock(boolean fair) {
+    this.coarseLock = new ReentrantReadWriteLock(fair);
+  }
+  
+  @Override
+  public Lock readLock() {
+    return coarseLock.readLock();
+  }
+  
+  @Override
+  public Lock writeLock() {
+    return coarseLock.writeLock();
+  }
+  
+  public int getReadHoldCount() {
+    return coarseLock.getReadHoldCount();
+  }
+  
+  public int getWriteHoldCount() {
+    return coarseLock.getWriteHoldCount();
+  }
+  
+  public boolean isWriteLockedByCurrentThread() {
+    return coarseLock.isWriteLockedByCurrentThread();
+  }
+}

+ 6 - 51
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -17,6 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
+
 import java.io.IOException;
 import java.io.PrintStream;
 import java.net.InetSocketAddress;
@@ -430,17 +434,11 @@ public class NameNode implements NameNodeStatusMXBean {
     return getHttpAddress(conf);
   }
 
-  /** @return the NameNode HTTP address set in the conf. */
+  /** @return the NameNode HTTP address. */
   public static InetSocketAddress getHttpAddress(Configuration conf) {
     return  NetUtils.createSocketAddr(
         conf.get(DFS_NAMENODE_HTTP_ADDRESS_KEY, DFS_NAMENODE_HTTP_ADDRESS_DEFAULT));
   }
-  
-  protected void setHttpServerAddress(Configuration conf) {
-    String hostPort = NetUtils.getHostPortString(getHttpAddress());
-    conf.set(DFS_NAMENODE_HTTP_ADDRESS_KEY, hostPort);
-    LOG.info("Web-server up at: " + hostPort);
-  }
 
   protected void loadNamesystem(Configuration conf) throws IOException {
     this.namesystem = FSNamesystem.loadFromDisk(conf);
@@ -490,7 +488,6 @@ public class NameNode implements NameNodeStatusMXBean {
 
     if (NamenodeRole.NAMENODE == role) {
       startHttpServer(conf);
-      validateConfigurationSettingsOrAbort(conf);
     }
     loadNamesystem(conf);
 
@@ -498,8 +495,6 @@ public class NameNode implements NameNodeStatusMXBean {
     if (NamenodeRole.NAMENODE == role) {
       httpServer.setNameNodeAddress(getNameNodeAddress());
       httpServer.setFSImage(getFSImage());
-    } else {
-      validateConfigurationSettingsOrAbort(conf);
     }
     
     pauseMonitor = new JvmPauseMonitor(conf);
@@ -517,45 +512,6 @@ public class NameNode implements NameNodeStatusMXBean {
     return new NameNodeRpcServer(conf, this);
   }
 
-  /**
-   * Verifies that the final Configuration Settings look ok for the NameNode to
-   * properly start up
-   * Things to check for include:
-   * - HTTP Server Port does not equal the RPC Server Port
-   * @param conf
-   * @throws IOException
-   */
-  protected void validateConfigurationSettings(final Configuration conf) 
-      throws IOException {
-    // check to make sure the web port and rpc port do not match 
-    if(getHttpServerAddress(conf).getPort() 
-        == getRpcServerAddress(conf).getPort()) {
-      String errMsg = "dfs.namenode.rpc-address " +
-          "("+ getRpcServerAddress(conf) + ") and " +
-          "dfs.namenode.http-address ("+ getHttpServerAddress(conf) + ") " +
-          "configuration keys are bound to the same port, unable to start " +
-          "NameNode. Port: " + getRpcServerAddress(conf).getPort();
-      throw new IOException(errMsg);
-    } 
-  }
-
-  /**
-   * Validate NameNode configuration.  Log a fatal error and abort if
-   * configuration is invalid.
-   * 
-   * @param conf Configuration to validate
-   * @throws IOException thrown if conf is invalid
-   */
-  private void validateConfigurationSettingsOrAbort(Configuration conf)
-      throws IOException {
-    try {
-      validateConfigurationSettings(conf);
-    } catch (IOException e) {
-      LOG.fatal(e.toString());
-      throw e;
-    }
-  }
-
   /** Start the services common to active and standby states */
   private void startCommonServices(Configuration conf) throws IOException {
     namesystem.startCommonServices(conf, haContext);
@@ -634,7 +590,6 @@ public class NameNode implements NameNodeStatusMXBean {
     httpServer = new NameNodeHttpServer(conf, this, getHttpServerAddress(conf));
     httpServer.start();
     httpServer.setStartupProgress(startupProgress);
-    setHttpServerAddress(conf);
   }
   
   private void stopHttpServer() {
@@ -656,7 +611,7 @@ public class NameNode implements NameNodeStatusMXBean {
    * <li>{@link StartupOption#CHECKPOINT CHECKPOINT} - start checkpoint node</li>
    * <li>{@link StartupOption#UPGRADE UPGRADE} - start the cluster  
    * upgrade and create a snapshot of the current file system state</li> 
-   * <li>{@link StartupOption#RECOVERY RECOVERY} - recover name node
+   * <li>{@link StartupOption#RECOVER RECOVERY} - recover name node
    * metadata</li>
    * <li>{@link StartupOption#ROLLBACK ROLLBACK} - roll the  
    *            cluster back to the previous state</li>

+ 95 - 53
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java

@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ADMIN;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_KEY;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -37,6 +39,7 @@ import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMetho
 import org.apache.hadoop.hdfs.web.AuthFilter;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.resources.Param;
+import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
@@ -60,76 +63,116 @@ public class NameNodeHttpServer {
   public static final String FSIMAGE_ATTRIBUTE_KEY = "name.system.image";
   protected static final String NAMENODE_ATTRIBUTE_KEY = "name.node";
   public static final String STARTUP_PROGRESS_ATTRIBUTE_KEY = "startup.progress";
-  
-  public NameNodeHttpServer(
-      Configuration conf,
-      NameNode nn,
+
+  NameNodeHttpServer(Configuration conf, NameNode nn,
       InetSocketAddress bindAddress) {
     this.conf = conf;
     this.nn = nn;
     this.bindAddress = bindAddress;
   }
-  
+
+  private void initWebHdfs(Configuration conf) throws IOException {
+    if (WebHdfsFileSystem.isEnabled(conf, HttpServer.LOG)) {
+      //add SPNEGO authentication filter for webhdfs
+      final String name = "SPNEGO";
+      final String classname = AuthFilter.class.getName();
+      final String pathSpec = WebHdfsFileSystem.PATH_PREFIX + "/*";
+      Map<String, String> params = getAuthFilterParams(conf);
+      HttpServer.defineFilter(httpServer.getWebAppContext(), name, classname, params,
+          new String[]{pathSpec});
+      HttpServer.LOG.info("Added filter '" + name + "' (class=" + classname + ")");
+
+      // add webhdfs packages
+      httpServer.addJerseyResourcePackage(
+          NamenodeWebHdfsMethods.class.getPackage().getName()
+              + ";" + Param.class.getPackage().getName(), pathSpec);
+    }
+  }
+
+  /**
+   * @see DFSUtil#getHttpPolicy(org.apache.hadoop.conf.Configuration)
+   * for information related to the different configuration options and
+   * Http Policy is decided.
+   */
   void start() throws IOException {
+    HttpConfig.Policy policy = DFSUtil.getHttpPolicy(conf);
     final String infoHost = bindAddress.getHostName();
-    int infoPort = bindAddress.getPort();
-    HttpServer.Builder builder = new HttpServer.Builder().setName("hdfs")
-        .addEndpoint(URI.create(("http://" + NetUtils.getHostPortString(bindAddress))))
-        .setFindPort(infoPort == 0).setConf(conf).setACL(
-            new AccessControlList(conf.get(DFS_ADMIN, " ")))
+
+    HttpServer.Builder builder = new HttpServer.Builder()
+        .setName("hdfs")
+        .setConf(conf)
+        .setACL(new AccessControlList(conf.get(DFS_ADMIN, " ")))
         .setSecurityEnabled(UserGroupInformation.isSecurityEnabled())
         .setUsernameConfKey(
             DFSConfigKeys.DFS_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY)
-        .setKeytabConfKey(DFSUtil.getSpnegoKeytabKey(conf,
-            DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY));
+        .setKeytabConfKey(
+            DFSUtil.getSpnegoKeytabKey(conf,
+                DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY));
 
-    boolean certSSL = conf.getBoolean(DFSConfigKeys.DFS_HTTPS_ENABLE_KEY, false);
-    if (certSSL) {
-      httpsAddress = NetUtils.createSocketAddr(conf.get(
+    if (policy.isHttpEnabled()) {
+      int port = bindAddress.getPort();
+      if (port == 0) {
+        builder.setFindPort(true);
+      }
+      builder.addEndpoint(URI.create("http://" + infoHost + ":" + port));
+    }
+
+    if (policy.isHttpsEnabled()) {
+      final String httpsAddrString = conf.get(
           DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY,
-          DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT));
+          DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT);
+      InetSocketAddress addr = NetUtils.createSocketAddr(httpsAddrString);
 
-      builder.addEndpoint(URI.create("https://"
-          + NetUtils.getHostPortString(httpsAddress)));
       Configuration sslConf = new Configuration(false);
-      sslConf.setBoolean(DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_KEY, conf
-          .getBoolean(DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_KEY,
-              DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT));
+
       sslConf.addResource(conf.get(
           DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY,
           DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_DEFAULT));
+
+      sslConf.addResource(conf.get(
+          DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY,
+          DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_DEFAULT));
+      sslConf.setBoolean(DFS_CLIENT_HTTPS_NEED_AUTH_KEY, conf.getBoolean(
+          DFS_CLIENT_HTTPS_NEED_AUTH_KEY, DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT));
       DFSUtil.loadSslConfToHttpServerBuilder(builder, sslConf);
+
+      if (addr.getPort() == 0) {
+        builder.setFindPort(true);
+      }
+
+      builder.addEndpoint(URI.create("https://"
+          + NetUtils.getHostPortString(addr)));
     }
 
     httpServer = builder.build();
-    if (WebHdfsFileSystem.isEnabled(conf, HttpServer.LOG)) {
-      //add SPNEGO authentication filter for webhdfs
-      final String name = "SPNEGO";
-      final String classname = AuthFilter.class.getName();
-      final String pathSpec = WebHdfsFileSystem.PATH_PREFIX + "/*";
-      Map<String, String> params = getAuthFilterParams(conf);
-      HttpServer.defineFilter(httpServer.getWebAppContext(), name, classname, params,
-          new String[]{pathSpec});
-      HttpServer.LOG.info("Added filter '" + name + "' (class=" + classname + ")");
 
-      // add webhdfs packages
-      httpServer.addJerseyResourcePackage(
-          NamenodeWebHdfsMethods.class.getPackage().getName()
-          + ";" + Param.class.getPackage().getName(), pathSpec);
-      }
+    if (policy.isHttpsEnabled()) {
+      // assume same ssl port for all datanodes
+      InetSocketAddress datanodeSslPort = NetUtils.createSocketAddr(conf.get(
+          DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY, infoHost + ":"
+              + DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT));
+      httpServer.setAttribute(DFSConfigKeys.DFS_DATANODE_HTTPS_PORT_KEY,
+          datanodeSslPort.getPort());
+    }
+
+    initWebHdfs(conf);
 
     httpServer.setAttribute(NAMENODE_ATTRIBUTE_KEY, nn);
     httpServer.setAttribute(JspHelper.CURRENT_CONF, conf);
     setupServlets(httpServer, conf);
     httpServer.start();
-    httpAddress = httpServer.getConnectorAddress(0);
-    if (certSSL) {
-      httpsAddress = httpServer.getConnectorAddress(1);
-      // assume same ssl port for all datanodes
-      InetSocketAddress datanodeSslPort = NetUtils.createSocketAddr(conf.get(
-        DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY, infoHost + ":" + 50475));
-      httpServer.setAttribute(DFSConfigKeys.DFS_DATANODE_HTTPS_PORT_KEY, datanodeSslPort
-        .getPort());
+
+    int connIdx = 0;
+    if (policy.isHttpEnabled()) {
+      httpAddress = httpServer.getConnectorAddress(connIdx++);
+      conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY,
+          NetUtils.getHostPortString(httpAddress));
+    }
+
+    if (policy.isHttpsEnabled()) {
+      httpsAddress = httpServer.getConnectorAddress(connIdx);
+      conf.set(DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY,
+          NetUtils.getHostPortString(httpsAddress));
     }
   }
   
@@ -165,18 +208,17 @@ public class NameNodeHttpServer {
     return params;
   }
 
-
-  public void stop() throws Exception {
+  void stop() throws Exception {
     if (httpServer != null) {
       httpServer.stop();
     }
   }
 
-  public InetSocketAddress getHttpAddress() {
+  InetSocketAddress getHttpAddress() {
     return httpAddress;
   }
 
-  public InetSocketAddress getHttpsAddress() {
+  InetSocketAddress getHttpsAddress() {
     return httpsAddress;
   }
 
@@ -185,7 +227,7 @@ public class NameNodeHttpServer {
    * 
    * @param fsImage FSImage to set
    */
-  public void setFSImage(FSImage fsImage) {
+  void setFSImage(FSImage fsImage) {
     httpServer.setAttribute(FSIMAGE_ATTRIBUTE_KEY, fsImage);
   }
 
@@ -194,7 +236,7 @@ public class NameNodeHttpServer {
    * 
    * @param nameNodeAddress InetSocketAddress to set
    */
-  public void setNameNodeAddress(InetSocketAddress nameNodeAddress) {
+  void setNameNodeAddress(InetSocketAddress nameNodeAddress) {
     httpServer.setAttribute(NAMENODE_ADDRESS_ATTRIBUTE_KEY,
         NetUtils.getConnectAddress(nameNodeAddress));
   }
@@ -204,7 +246,7 @@ public class NameNodeHttpServer {
    * 
    * @param prog StartupProgress to set
    */
-  public void setStartupProgress(StartupProgress prog) {
+  void setStartupProgress(StartupProgress prog) {
     httpServer.setAttribute(STARTUP_PROGRESS_ATTRIBUTE_KEY, prog);
   }
 
@@ -234,7 +276,7 @@ public class NameNodeHttpServer {
         ContentSummaryServlet.class, false);
   }
 
-  public static FSImage getFsImageFromContext(ServletContext context) {
+  static FSImage getFsImageFromContext(ServletContext context) {
     return (FSImage)context.getAttribute(FSIMAGE_ATTRIBUTE_KEY);
   }
 
@@ -242,7 +284,7 @@ public class NameNodeHttpServer {
     return (NameNode)context.getAttribute(NAMENODE_ATTRIBUTE_KEY);
   }
 
-  public static Configuration getConfFromContext(ServletContext context) {
+  static Configuration getConfFromContext(ServletContext context) {
     return (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
   }
 
@@ -258,7 +300,7 @@ public class NameNodeHttpServer {
    * @param context ServletContext to get
    * @return StartupProgress associated with context
    */
-  public static StartupProgress getStartupProgressFromContext(
+  static StartupProgress getStartupProgressFromContext(
       ServletContext context) {
     return (StartupProgress)context.getAttribute(STARTUP_PROGRESS_ATTRIBUTE_KEY);
   }

+ 5 - 48
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -36,7 +36,6 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.BatchedRemoteIterator;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
@@ -46,8 +45,8 @@ import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.ha.HAServiceStatus;
@@ -1254,36 +1253,13 @@ class NameNodeRpcServer implements NamenodeProtocols {
     namesystem.removeCacheDirective(id);
   }
 
-  private class ServerSideCacheEntriesIterator 
-      extends BatchedRemoteIterator<Long, CacheDirectiveEntry> {
-
-    private final CacheDirectiveInfo filter;
-    
-    public ServerSideCacheEntriesIterator (Long firstKey, 
-        CacheDirectiveInfo filter) {
-      super(firstKey);
-      this.filter = filter;
-    }
-
-    @Override
-    public BatchedEntries<CacheDirectiveEntry> makeRequest(
-        Long nextKey) throws IOException {
-      return namesystem.listCacheDirectives(nextKey, filter);
-    }
-
-    @Override
-    public Long elementToPrevKey(CacheDirectiveEntry entry) {
-      return entry.getInfo().getId();
-    }
-  }
-  
   @Override
-  public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(long prevId,
+  public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(long prevId,
       CacheDirectiveInfo filter) throws IOException {
     if (filter == null) {
       filter = new CacheDirectiveInfo.Builder().build();
     }
-    return new ServerSideCacheEntriesIterator(prevId, filter);
+    return namesystem.listCacheDirectives(prevId, filter);
   }
 
   @Override
@@ -1301,29 +1277,10 @@ class NameNodeRpcServer implements NamenodeProtocols {
     namesystem.removeCachePool(cachePoolName);
   }
 
-  private class ServerSideCachePoolIterator 
-      extends BatchedRemoteIterator<String, CachePoolEntry> {
-
-    public ServerSideCachePoolIterator(String prevKey) {
-      super(prevKey);
-    }
-
-    @Override
-    public BatchedEntries<CachePoolEntry> makeRequest(String prevKey)
-        throws IOException {
-      return namesystem.listCachePools(prevKey);
-    }
-
-    @Override
-    public String elementToPrevKey(CachePoolEntry entry) {
-      return entry.getInfo().getPoolName();
-    }
-  }
-
   @Override
-  public RemoteIterator<CachePoolEntry> listCachePools(String prevKey)
+  public BatchedEntries<CachePoolEntry> listCachePools(String prevKey)
       throws IOException {
-    return new ServerSideCachePoolIterator(prevKey);
+    return namesystem.listCachePools(prevKey != null ? prevKey : "");
   }
 }
 

+ 2 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -30,7 +30,6 @@ import java.io.FilenameFilter;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
-import java.net.URISyntaxException;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
@@ -257,12 +256,7 @@ public class SecondaryNameNode implements Runnable {
 
     // initialize the webserver for uploading files.
     int tmpInfoPort = infoSocAddr.getPort();
-    URI httpEndpoint;
-    try {
-      httpEndpoint = new URI("http://" + NetUtils.getHostPortString(infoSocAddr));
-    } catch (URISyntaxException e) {
-      throw new IOException(e);
-    }
+    URI httpEndpoint = URI.create("http://" + NetUtils.getHostPortString(infoSocAddr));
 
     infoServer = new HttpServer.Builder().setName("secondary")
         .addEndpoint(httpEndpoint)
@@ -273,6 +267,7 @@ public class SecondaryNameNode implements Runnable {
             DFSConfigKeys.DFS_SECONDARY_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY)
         .setKeytabConfKey(DFSUtil.getSpnegoKeytabKey(conf,
             DFSConfigKeys.DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY)).build();
+
     infoServer.setAttribute("secondary.name.node", this);
     infoServer.setAttribute("name.system.image", checkpointImage);
     infoServer.setAttribute(JspHelper.CURRENT_CONF, conf);

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -249,8 +249,12 @@ public class NamenodeWebHdfsMethods {
         + Param.toSortedString("&", parameters);
     final String uripath = WebHdfsFileSystem.PATH_PREFIX + path;
 
-    final URI uri = new URI("http", null, dn.getHostName(), dn.getInfoPort(),
-        uripath, query, null);
+    final String scheme = request.getScheme();
+    int port = "http".equals(scheme) ? dn.getInfoPort() : dn
+        .getInfoSecurePort();
+    final URI uri = new URI(scheme, null, dn.getHostName(), port, uripath,
+        query, null);
+
     if (LOG.isTraceEnabled()) {
       LOG.trace("redirectURI=" + uri);
     }

+ 36 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolStats;
 import org.apache.hadoop.hdfs.server.namenode.CachePool;
 import org.apache.hadoop.hdfs.tools.TableListing.Justification;
 import org.apache.hadoop.ipc.RemoteException;
@@ -477,9 +478,10 @@ public class CacheAdmin extends Configured implements Tool {
           addField("EXPIRY", Justification.LEFT).
           addField("PATH", Justification.LEFT);
       if (printStats) {
-        tableBuilder.addField("NEEDED", Justification.RIGHT).
-                    addField("CACHED", Justification.RIGHT).
-                    addField("FILES", Justification.RIGHT);
+        tableBuilder.addField("BYTES_NEEDED", Justification.RIGHT).
+                    addField("BYTES_CACHED", Justification.RIGHT).
+                    addField("FILES_NEEDED", Justification.RIGHT).
+                    addField("FILES_CACHED", Justification.RIGHT);
       }
       TableListing tableListing = tableBuilder.build();
 
@@ -507,7 +509,8 @@ public class CacheAdmin extends Configured implements Tool {
         if (printStats) {
           row.add("" + stats.getBytesNeeded());
           row.add("" + stats.getBytesCached());
-          row.add("" + stats.getFilesAffected());
+          row.add("" + stats.getFilesNeeded());
+          row.add("" + stats.getFilesCached());
         }
         tableListing.addRow(row.toArray(new String[0]));
         numEntries++;
@@ -769,13 +772,14 @@ public class CacheAdmin extends Configured implements Tool {
 
     @Override
     public String getShortUsage() {
-      return "[" + getName() + " [name]]\n";
+      return "[" + getName() + " [-stats] [<name>]]\n";
     }
 
     @Override
     public String getLongUsage() {
       TableListing listing = getOptionDescriptionListing();
-      listing.addRow("[name]", "If specified, list only the named cache pool.");
+      listing.addRow("-stats", "Display additional cache pool statistics.");
+      listing.addRow("<name>", "If specified, list only the named cache pool.");
 
       return getShortUsage() + "\n" +
           WordUtils.wrap("Display information about one or more cache pools, " +
@@ -787,6 +791,7 @@ public class CacheAdmin extends Configured implements Tool {
     @Override
     public int run(Configuration conf, List<String> args) throws IOException {
       String name = StringUtils.popFirstNonOption(args);
+      final boolean printStats = StringUtils.popOption("-stats", args);
       if (!args.isEmpty()) {
         System.err.print("Can't understand arguments: " +
           Joiner.on(" ").join(args) + "\n");
@@ -794,28 +799,42 @@ public class CacheAdmin extends Configured implements Tool {
         return 1;
       }
       DistributedFileSystem dfs = getDFS(conf);
-      TableListing listing = new TableListing.Builder().
+      TableListing.Builder builder = new TableListing.Builder().
           addField("NAME", Justification.LEFT).
           addField("OWNER", Justification.LEFT).
           addField("GROUP", Justification.LEFT).
           addField("MODE", Justification.LEFT).
-          addField("WEIGHT", Justification.RIGHT).
-          build();
+          addField("WEIGHT", Justification.RIGHT);
+      if (printStats) {
+        builder.
+            addField("BYTES_NEEDED", Justification.RIGHT).
+            addField("BYTES_CACHED", Justification.RIGHT).
+            addField("FILES_NEEDED", Justification.RIGHT).
+            addField("FILES_CACHED", Justification.RIGHT);
+      }
+      TableListing listing = builder.build();
       int numResults = 0;
       try {
         RemoteIterator<CachePoolEntry> iter = dfs.listCachePools();
         while (iter.hasNext()) {
           CachePoolEntry entry = iter.next();
           CachePoolInfo info = entry.getInfo();
-          String[] row = new String[5];
+          LinkedList<String> row = new LinkedList<String>();
           if (name == null || info.getPoolName().equals(name)) {
-            row[0] = info.getPoolName();
-            row[1] = info.getOwnerName();
-            row[2] = info.getGroupName();
-            row[3] = info.getMode() != null ? info.getMode().toString() : null;
-            row[4] =
-                info.getWeight() != null ? info.getWeight().toString() : null;
-            listing.addRow(row);
+            row.add(info.getPoolName());
+            row.add(info.getOwnerName());
+            row.add(info.getGroupName());
+            row.add(info.getMode() != null ? info.getMode().toString() : null);
+            row.add(
+                info.getWeight() != null ? info.getWeight().toString() : null);
+            if (printStats) {
+              CachePoolStats stats = entry.getStats();
+              row.add(Long.toString(stats.getBytesNeeded()));
+              row.add(Long.toString(stats.getBytesCached()));
+              row.add(Long.toString(stats.getFilesNeeded()));
+              row.add(Long.toString(stats.getFilesCached()));
+            }
+            listing.addRow(row.toArray(new String[] {}));
             ++numResults;
             if (name != null) {
               break;

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -380,8 +380,9 @@ message CacheDirectiveInfoExpirationProto {
 message CacheDirectiveStatsProto {
   required int64 bytesNeeded = 1;
   required int64 bytesCached = 2;
-  required int64 filesAffected = 3;
-  required bool hasExpired = 4;
+  required int64 filesNeeded = 3;
+  required int64 filesCached = 4;
+  required bool hasExpired = 5;
 }
 
 message AddCacheDirectiveRequestProto {
@@ -432,7 +433,8 @@ message CachePoolInfoProto {
 message CachePoolStatsProto {
   required int64 bytesNeeded = 1;
   required int64 bytesCached = 2;
-  required int64 filesAffected = 3;
+  required int64 filesNeeded = 3;
+  required int64 filesCached = 4;
 }
 
 message AddCachePoolRequestProto {

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -137,7 +137,20 @@
 <property>
   <name>dfs.https.enable</name>
   <value>false</value>
+  <description>
+    Deprecated. Use "dfs.http.policy" instead.
+  </description>
+</property>
+
+<property>
+  <name>dfs.http.policy</name>
+  <value>HTTP_ONLY</value>
   <description>Decide if HTTPS(SSL) is supported on HDFS
+    This configures the HTTP endpoint for HDFS daemons:
+      The following values are supported:
+      - HTTP_ONLY : Service is provided only on http
+      - HTTPS_ONLY : Service is provided only on https
+      - HTTP_AND_HTTPS : Service is provided both on http and https
   </description>
 </property>
 

+ 13 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -33,6 +33,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HOSTS;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY;
@@ -906,12 +907,17 @@ public class MiniDFSCluster {
     
     // After the NN has started, set back the bound ports into
     // the conf
-    conf.set(DFSUtil.addKeySuffixes(
-        DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId, nnId),
-        nn.getNameNodeAddressHostPortString());
-    conf.set(DFSUtil.addKeySuffixes(
-        DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId, nnId), NetUtils
-        .getHostPortString(nn.getHttpAddress()));
+    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
+        nameserviceId, nnId), nn.getNameNodeAddressHostPortString());
+    if (nn.getHttpAddress() != null) {
+      conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_HTTP_ADDRESS_KEY,
+          nameserviceId, nnId), NetUtils.getHostPortString(nn.getHttpAddress()));
+    }
+    if (nn.getHttpsAddress() != null) {
+      conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_HTTPS_ADDRESS_KEY,
+          nameserviceId, nnId), NetUtils.getHostPortString(nn.getHttpsAddress()));
+    }
+
     DFSUtil.setGenericConf(conf, nameserviceId, nnId,
         DFS_NAMENODE_HTTP_ADDRESS_KEY);
     nameNodes[nnIndex] = new NameNodeInfo(nn, nameserviceId, nnId,
@@ -1188,9 +1194,8 @@ public class MiniDFSCluster {
 
       SecureResources secureResources = null;
       if (UserGroupInformation.isSecurityEnabled()) {
-        SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, dnConf);
         try {
-          secureResources = SecureDataNodeStarter.getSecureResources(sslFactory, dnConf);
+          secureResources = SecureDataNodeStarter.getSecureResources(dnConf);
         } catch (Exception ex) {
           ex.printStackTrace();
         }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java

@@ -158,9 +158,8 @@ public class MiniDFSClusterWithNodeGroup extends MiniDFSCluster {
       
       SecureResources secureResources = null;
       if (UserGroupInformation.isSecurityEnabled()) {
-        SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, dnConf);
         try {
-          secureResources = SecureDataNodeStarter.getSecureResources(sslFactory, dnConf);
+          secureResources = SecureDataNodeStarter.getSecureResources(dnConf);
         } catch (Exception ex) {
           ex.printStackTrace();
         }

+ 130 - 40
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java

@@ -57,10 +57,12 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveIterator;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo.Expiration;
+import org.apache.hadoop.hdfs.protocol.CachePoolStats;
 import org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@@ -622,45 +624,111 @@ public class TestCacheDirectives {
     }, 500, 60000);
   }
 
-  private static void waitForCachedStats(final DistributedFileSystem dfs,
-      final long targetFilesAffected, final long targetBytesNeeded,
-        final long targetBytesCached,
-          final CacheDirectiveInfo filter, final String infoString)
+  private static void waitForCacheDirectiveStats(final DistributedFileSystem dfs,
+      final long targetBytesNeeded, final long targetBytesCached,
+      final long targetFilesNeeded, final long targetFilesCached,
+      final CacheDirectiveInfo filter, final String infoString)
             throws Exception {
-      LOG.info("Polling listDirectives{" + 
-          ((filter == null) ? "ALL" : filter.toString()) +
-          " for " + targetFilesAffected + " targetFilesAffected, " +
-          targetBytesNeeded + " targetBytesNeeded, " +
-          targetBytesCached + " targetBytesCached");
-      GenericTestUtils.waitFor(new Supplier<Boolean>() {
-        @Override
-        public Boolean get() {
-          RemoteIterator<CacheDirectiveEntry> iter = null;
-          CacheDirectiveEntry entry = null;
+    LOG.info("Polling listCacheDirectives " + 
+        ((filter == null) ? "ALL" : filter.toString()) + " for " +
+        targetBytesNeeded + " targetBytesNeeded, " +
+        targetBytesCached + " targetBytesCached, " +
+        targetFilesNeeded + " targetFilesNeeded, " +
+        targetFilesCached + " targetFilesCached");
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        RemoteIterator<CacheDirectiveEntry> iter = null;
+        CacheDirectiveEntry entry = null;
+        try {
+          iter = dfs.listCacheDirectives(filter);
+          entry = iter.next();
+        } catch (IOException e) {
+          fail("got IOException while calling " +
+              "listCacheDirectives: " + e.getMessage());
+        }
+        Assert.assertNotNull(entry);
+        CacheDirectiveStats stats = entry.getStats();
+        if ((targetBytesNeeded == stats.getBytesNeeded()) &&
+            (targetBytesCached == stats.getBytesCached()) &&
+            (targetFilesNeeded == stats.getFilesNeeded()) &&
+            (targetFilesCached == stats.getFilesCached())) {
+          return true;
+        } else {
+          LOG.info(infoString + ": " +
+              "filesNeeded: " +
+              stats.getFilesNeeded() + "/" + targetFilesNeeded +
+              ", filesCached: " + 
+              stats.getFilesCached() + "/" + targetFilesCached +
+              ", bytesNeeded: " +
+              stats.getBytesNeeded() + "/" + targetBytesNeeded +
+              ", bytesCached: " + 
+              stats.getBytesCached() + "/" + targetBytesCached);
+          return false;
+        }
+      }
+    }, 500, 60000);
+  }
+
+  private static void waitForCachePoolStats(final DistributedFileSystem dfs,
+      final long targetBytesNeeded, final long targetBytesCached,
+      final long targetFilesNeeded, final long targetFilesCached,
+      final CachePoolInfo pool, final String infoString)
+            throws Exception {
+    LOG.info("Polling listCachePools " + pool.toString() + " for " +
+        targetBytesNeeded + " targetBytesNeeded, " +
+        targetBytesCached + " targetBytesCached, " +
+        targetFilesNeeded + " targetFilesNeeded, " +
+        targetFilesCached + " targetFilesCached");
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        RemoteIterator<CachePoolEntry> iter = null;
+        try {
+          iter = dfs.listCachePools();
+        } catch (IOException e) {
+          fail("got IOException while calling " +
+              "listCachePools: " + e.getMessage());
+        }
+        while (true) {
+          CachePoolEntry entry = null;
           try {
-            iter = dfs.listCacheDirectives(filter);
+            if (!iter.hasNext()) {
+              break;
+            }
             entry = iter.next();
           } catch (IOException e) {
-            fail("got IOException while calling " +
-                "listCacheDirectives: " + e.getMessage());
+            fail("got IOException while iterating through " +
+                "listCachePools: " + e.getMessage());
+          }
+          if (entry == null) {
+            break;
+          }
+          if (!entry.getInfo().getPoolName().equals(pool.getPoolName())) {
+            continue;
           }
-          Assert.assertNotNull(entry);
-          CacheDirectiveStats stats = entry.getStats();
-          if ((targetFilesAffected == stats.getFilesAffected()) &&
-              (targetBytesNeeded == stats.getBytesNeeded()) &&
-              (targetBytesCached == stats.getBytesCached())) {
+          CachePoolStats stats = entry.getStats();
+          if ((targetBytesNeeded == stats.getBytesNeeded()) &&
+              (targetBytesCached == stats.getBytesCached()) &&
+              (targetFilesNeeded == stats.getFilesNeeded()) &&
+              (targetFilesCached == stats.getFilesCached())) {
             return true;
           } else {
-            LOG.info(infoString + ": filesAffected: " + 
-              stats.getFilesAffected() + "/" + targetFilesAffected +
-              ", bytesNeeded: " +
+            LOG.info(infoString + ": " +
+                "filesNeeded: " +
+                stats.getFilesNeeded() + "/" + targetFilesNeeded +
+                ", filesCached: " + 
+                stats.getFilesCached() + "/" + targetFilesCached +
+                ", bytesNeeded: " +
                 stats.getBytesNeeded() + "/" + targetBytesNeeded +
-              ", bytesCached: " + 
+                ", bytesCached: " + 
                 stats.getBytesCached() + "/" + targetBytesCached);
             return false;
           }
         }
-      }, 500, 60000);
+        return false;
+      }
+    }, 500, 60000);
   }
 
   private static void checkNumCachedReplicas(final DistributedFileSystem dfs,
@@ -763,7 +831,7 @@ public class TestCacheDirectives {
       }
       // Uncache and check each path in sequence
       RemoteIterator<CacheDirectiveEntry> entries =
-          nnRpc.listCacheDirectives(0, null);
+        new CacheDirectiveIterator(nnRpc, null);
       for (int i=0; i<numFiles; i++) {
         CacheDirectiveEntry entry = entries.next();
         nnRpc.removeCacheDirective(entry.getInfo().getId());
@@ -836,7 +904,8 @@ public class TestCacheDirectives {
       NameNode namenode = cluster.getNameNode();
       // Create the pool
       final String pool = "friendlyPool";
-      dfs.addCachePool(new CachePoolInfo(pool));
+      final CachePoolInfo poolInfo = new CachePoolInfo(pool);
+      dfs.addCachePool(poolInfo);
       // Create some test files
       final List<Path> paths = new LinkedList<Path>();
       paths.add(new Path("/foo/bar"));
@@ -852,6 +921,7 @@ public class TestCacheDirectives {
       }
       waitForCachedBlocks(namenode, 0, 0,
           "testWaitForCachedReplicasInDirectory:0");
+
       // cache entire directory
       long id = dfs.addCacheDirective(
             new CacheDirectiveInfo.Builder().
@@ -860,14 +930,20 @@ public class TestCacheDirectives {
               setPool(pool).
               build());
       waitForCachedBlocks(namenode, 4, 8,
-          "testWaitForCachedReplicasInDirectory:1");
+          "testWaitForCachedReplicasInDirectory:1:blocks");
       // Verify that listDirectives gives the stats we want.
-      waitForCachedStats(dfs, 2,
-          8 * BLOCK_SIZE, 8 * BLOCK_SIZE,
+      waitForCacheDirectiveStats(dfs,
+          4 * numBlocksPerFile * BLOCK_SIZE, 4 * numBlocksPerFile * BLOCK_SIZE,
+          2, 2,
           new CacheDirectiveInfo.Builder().
               setPath(new Path("/foo")).
               build(),
-          "testWaitForCachedReplicasInDirectory:2");
+          "testWaitForCachedReplicasInDirectory:1:directive");
+      waitForCachePoolStats(dfs,
+          4 * numBlocksPerFile * BLOCK_SIZE, 4 * numBlocksPerFile * BLOCK_SIZE,
+          2, 2,
+          poolInfo, "testWaitForCachedReplicasInDirectory:1:pool");
+
       long id2 = dfs.addCacheDirective(
             new CacheDirectiveInfo.Builder().
               setPath(new Path("/foo/bar")).
@@ -876,28 +952,42 @@ public class TestCacheDirectives {
               build());
       // wait for an additional 2 cached replicas to come up
       waitForCachedBlocks(namenode, 4, 10,
-          "testWaitForCachedReplicasInDirectory:3");
+          "testWaitForCachedReplicasInDirectory:2:blocks");
       // the directory directive's stats are unchanged
-      waitForCachedStats(dfs, 2,
-          8 * BLOCK_SIZE, 8 * BLOCK_SIZE,
+      waitForCacheDirectiveStats(dfs,
+          4 * numBlocksPerFile * BLOCK_SIZE, 4 * numBlocksPerFile * BLOCK_SIZE,
+          2, 2,
           new CacheDirectiveInfo.Builder().
               setPath(new Path("/foo")).
               build(),
-          "testWaitForCachedReplicasInDirectory:4");
+          "testWaitForCachedReplicasInDirectory:2:directive-1");
       // verify /foo/bar's stats
-      waitForCachedStats(dfs, 1,
+      waitForCacheDirectiveStats(dfs,
           4 * numBlocksPerFile * BLOCK_SIZE,
           // only 3 because the file only has 3 replicas, not 4 as requested.
           3 * numBlocksPerFile * BLOCK_SIZE,
+          1,
+          // only 0 because the file can't be fully cached
+          0,
           new CacheDirectiveInfo.Builder().
               setPath(new Path("/foo/bar")).
               build(),
-          "testWaitForCachedReplicasInDirectory:5");
+          "testWaitForCachedReplicasInDirectory:2:directive-2");
+      waitForCachePoolStats(dfs,
+          (4+4) * numBlocksPerFile * BLOCK_SIZE,
+          (4+3) * numBlocksPerFile * BLOCK_SIZE,
+          3, 2,
+          poolInfo, "testWaitForCachedReplicasInDirectory:2:pool");
+
       // remove and watch numCached go to 0
       dfs.removeCacheDirective(id);
       dfs.removeCacheDirective(id2);
       waitForCachedBlocks(namenode, 0, 0,
-          "testWaitForCachedReplicasInDirectory:6");
+          "testWaitForCachedReplicasInDirectory:3:blocks");
+      waitForCachePoolStats(dfs,
+          0, 0,
+          0, 0,
+          poolInfo, "testWaitForCachedReplicasInDirectory:3:pool");
     } finally {
       cluster.shutdown();
     }

+ 36 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java

@@ -158,4 +158,40 @@ public class TestFSNamesystem {
     fsNamesystem = new FSNamesystem(conf, fsImage);
     assertFalse(fsNamesystem.getFsLockForTests().isFair());
   }  
+  
+  @Test
+  public void testFSNamesystemLockCompatibility() {
+    FSNamesystemLock rwLock = new FSNamesystemLock(true);
+
+    assertEquals(0, rwLock.getReadHoldCount());
+    rwLock.readLock().lock();
+    assertEquals(1, rwLock.getReadHoldCount());
+
+    rwLock.readLock().lock();
+    assertEquals(2, rwLock.getReadHoldCount());
+
+    rwLock.readLock().unlock();
+    assertEquals(1, rwLock.getReadHoldCount());
+
+    rwLock.readLock().unlock();
+    assertEquals(0, rwLock.getReadHoldCount());
+
+    assertFalse(rwLock.isWriteLockedByCurrentThread());
+    assertEquals(0, rwLock.getWriteHoldCount());
+    rwLock.writeLock().lock();
+    assertTrue(rwLock.isWriteLockedByCurrentThread());
+    assertEquals(1, rwLock.getWriteHoldCount());
+    
+    rwLock.writeLock().lock();
+    assertTrue(rwLock.isWriteLockedByCurrentThread());
+    assertEquals(2, rwLock.getWriteHoldCount());
+
+    rwLock.writeLock().unlock();
+    assertTrue(rwLock.isWriteLockedByCurrentThread());
+    assertEquals(1, rwLock.getWriteHoldCount());
+
+    rwLock.writeLock().unlock();
+    assertFalse(rwLock.isWriteLockedByCurrentThread());
+    assertEquals(0, rwLock.getWriteHoldCount());
+  }
 }

+ 127 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeHttpServer.java

@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.File;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.net.URLConnection;
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.web.URLConnectionFactory;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.http.HttpConfig.Policy;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(value = Parameterized.class)
+public class TestNameNodeHttpServer {
+  private static final String BASEDIR = System.getProperty("test.build.dir",
+      "target/test-dir") + "/" + TestNameNodeHttpServer.class.getSimpleName();
+  private static String keystoresDir;
+  private static String sslConfDir;
+  private static Configuration conf;
+  private static URLConnectionFactory connectionFactory;
+
+  @Parameters
+  public static Collection<Object[]> policy() {
+    Object[][] params = new Object[][] { { HttpConfig.Policy.HTTP_ONLY },
+        { HttpConfig.Policy.HTTPS_ONLY }, { HttpConfig.Policy.HTTP_AND_HTTPS } };
+    return Arrays.asList(params);
+  }
+
+  private final HttpConfig.Policy policy;
+
+  public TestNameNodeHttpServer(Policy policy) {
+    super();
+    this.policy = policy;
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    File base = new File(BASEDIR);
+    FileUtil.fullyDelete(base);
+    base.mkdirs();
+    conf = new Configuration();
+    keystoresDir = new File(BASEDIR).getAbsolutePath();
+    sslConfDir = KeyStoreTestUtil.getClasspathDir(TestNameNodeHttpServer.class);
+    KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
+    connectionFactory = URLConnectionFactory
+        .newDefaultURLConnectionFactory(conf);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    FileUtil.fullyDelete(new File(BASEDIR));
+    KeyStoreTestUtil.cleanupSSLConfig(keystoresDir, sslConfDir);
+  }
+
+  @Test
+  public void testHttpPolicy() throws Exception {
+    conf.set(DFSConfigKeys.DFS_HTTP_POLICY_KEY, policy.name());
+
+    InetSocketAddress addr = InetSocketAddress.createUnresolved("localhost", 0);
+    NameNodeHttpServer server = null;
+    try {
+      server = new NameNodeHttpServer(conf, null, addr);
+      server.start();
+
+      Assert.assertTrue(implies(policy.isHttpEnabled(),
+          canAccess("http", server.getHttpAddress())));
+      Assert.assertTrue(implies(!policy.isHttpEnabled(),
+          server.getHttpAddress() == null));
+
+      Assert.assertTrue(implies(policy.isHttpsEnabled(),
+          canAccess("https", server.getHttpsAddress())));
+      Assert.assertTrue(implies(!policy.isHttpsEnabled(),
+          server.getHttpsAddress() == null));
+
+    } finally {
+      server.stop();
+    }
+  }
+
+  private static boolean canAccess(String scheme, InetSocketAddress addr) {
+    if (addr == null)
+      return false;
+    try {
+      URL url = new URL(scheme + "://" + NetUtils.getHostPortString(addr));
+      URLConnection conn = connectionFactory.openConnection(url);
+      conn.connect();
+      conn.getContent();
+    } catch (Exception e) {
+      return false;
+    }
+    return true;
+  }
+
+  private static boolean implies(boolean a, boolean b) {
+    return !a || b;
+  }
+}

+ 3 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java

@@ -22,6 +22,7 @@ import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.BindException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -50,7 +51,7 @@ public class TestValidateConfigurationSettings {
    * an exception
    * is thrown when trying to re-use the same port
    */
-  @Test
+  @Test(expected = BindException.class)
   public void testThatMatchingRPCandHttpPortsThrowException() 
       throws IOException {
 
@@ -63,14 +64,7 @@ public class TestValidateConfigurationSettings {
     FileSystem.setDefaultUri(conf, "hdfs://localhost:9000"); 
     conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:9000");
     DFSTestUtil.formatNameNode(conf);
-    try {
-      NameNode nameNode = new NameNode(conf);
-      fail("Should have throw the exception since the ports match");
-    } catch (IOException e) {
-      // verify we're getting the right IOException
-      assertTrue(e.toString().contains("dfs.namenode.rpc-address (")); 
-      System.out.println("Got expected exception: " + e.toString());
-    }
+    new NameNode(conf);
   }
 
   /**

+ 92 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java

@@ -29,6 +29,7 @@ import java.net.URI;
 import java.net.UnknownHostException;
 import java.util.EnumSet;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Random;
@@ -86,6 +87,7 @@ public class TestRetryCacheWithHA {
   private static final int BlockSize = 1024;
   private static final short DataNodes = 3;
   private static final int CHECKTIMES = 10;
+  private static final int ResponseSize = 3;
   
   private MiniDFSCluster cluster;
   private DistributedFileSystem dfs;
@@ -120,6 +122,8 @@ public class TestRetryCacheWithHA {
   @Before
   public void setup() throws Exception {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BlockSize);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DIRECTIVES_NUM_RESPONSES, ResponseSize);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES, ResponseSize);
     cluster = new MiniDFSCluster.Builder(conf)
         .nnTopology(MiniDFSNNTopology.simpleHATopology())
         .numDataNodes(DataNodes).build();
@@ -1176,4 +1180,92 @@ public class TestRetryCacheWithHA {
           + results.get(op.name));
     }
   }
+
+  /**
+   * Add a list of cache pools, list cache pools,
+   * switch active NN, and list cache pools again.
+   */
+  @Test (timeout=60000)
+  public void testListCachePools() throws Exception {
+    final int poolCount = 7;
+    HashSet<String> poolNames = new HashSet<String>(poolCount);
+    for (int i=0; i<poolCount; i++) {
+      String poolName = "testListCachePools-" + i;
+      dfs.addCachePool(new CachePoolInfo(poolName));
+      poolNames.add(poolName);
+    }
+    listCachePools(poolNames, 0);
+
+    cluster.transitionToStandby(0);
+    cluster.transitionToActive(1);
+    cluster.waitActive(1);
+    listCachePools(poolNames, 1);
+  }
+
+  /**
+   * Add a list of cache directives, list cache directives,
+   * switch active NN, and list cache directives again.
+   */
+  @Test (timeout=60000)
+  public void testListCacheDirectives() throws Exception {
+    final int poolCount = 7;
+    HashSet<String> poolNames = new HashSet<String>(poolCount);
+    Path path = new Path("/p");
+    for (int i=0; i<poolCount; i++) {
+      String poolName = "testListCacheDirectives-" + i;
+      CacheDirectiveInfo directiveInfo =
+        new CacheDirectiveInfo.Builder().setPool(poolName).setPath(path).build();
+      dfs.addCachePool(new CachePoolInfo(poolName));
+      dfs.addCacheDirective(directiveInfo);
+      poolNames.add(poolName);
+    }
+    listCacheDirectives(poolNames, 0);
+
+    cluster.transitionToStandby(0);
+    cluster.transitionToActive(1);
+    cluster.waitActive(1);
+    listCacheDirectives(poolNames, 1);
+  }
+
+  @SuppressWarnings("unchecked")
+  private void listCachePools(
+      HashSet<String> poolNames, int active) throws Exception {
+    HashSet<String> tmpNames = (HashSet<String>)poolNames.clone();
+    RemoteIterator<CachePoolEntry> pools = dfs.listCachePools();
+    int poolCount = poolNames.size();
+    for (int i=0; i<poolCount; i++) {
+      CachePoolEntry pool = pools.next();
+      String pollName = pool.getInfo().getPoolName();
+      assertTrue("The pool name should be expected", tmpNames.remove(pollName));
+      if (i % 2 == 0) {
+        int standby = active;
+        active = (standby == 0) ? 1 : 0;
+        cluster.transitionToStandby(standby);
+        cluster.transitionToActive(active);
+        cluster.waitActive(active);
+      }
+    }
+    assertTrue("All pools must be found", tmpNames.isEmpty());
+  }
+
+  @SuppressWarnings("unchecked")
+  private void listCacheDirectives(
+      HashSet<String> poolNames, int active) throws Exception {
+    HashSet<String> tmpNames = (HashSet<String>)poolNames.clone();
+    RemoteIterator<CacheDirectiveEntry> directives = dfs.listCacheDirectives(null);
+    int poolCount = poolNames.size();
+    for (int i=0; i<poolCount; i++) {
+      CacheDirectiveEntry directive = directives.next();
+      String pollName = directive.getInfo().getPool();
+      assertTrue("The pool name should be expected", tmpNames.remove(pollName));
+      if (i % 2 == 0) {
+        int standby = active;
+        active = (standby == 0) ? 1 : 0;
+        cluster.transitionToStandby(standby);
+        cluster.transitionToActive(active);
+        cluster.waitActive(active);
+      }
+    }
+    assertTrue("All pools must be found", tmpNames.isEmpty());
+  }
 }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHttpsFileSystem.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -49,7 +50,7 @@ public class TestHttpsFileSystem {
   public static void setUp() throws Exception {
     conf = new Configuration();
     conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
-    conf.setBoolean(DFSConfigKeys.DFS_HTTPS_ENABLE_KEY, true);
+    conf.set(DFSConfigKeys.DFS_HTTP_POLICY_KEY, HttpConfig.Policy.HTTPS_ONLY.name());
     conf.set(DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY, "localhost:0");
 
     File base = new File(BASEDIR);

+ 58 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCacheAdminConf.xml

@@ -399,5 +399,63 @@
       </comparators>
     </test>
 
+    <test> <!--Tested -->
+      <description>Testing listing cache pool statistics</description>
+      <test-commands>
+        <cache-admin-command>-addPool foo -owner bob -group bob -mode 0664</cache-admin-command>
+        <cache-admin-command>-addPool bar -owner alice -group alicegroup -mode 0755</cache-admin-command>
+        <cache-admin-command>-listPools -stats</cache-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <cache-admin-command>-removePool foo</cache-admin-command>
+        <cache-admin-command>-removePool bar</cache-admin-command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Found 2 results.</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>bar   alice  alicegroup  rwxr-xr-x      100             0             0             0             0</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>foo   bob    bob         rw-rw-r--      100             0             0             0             0</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!--Tested -->
+      <description>Testing listing cache directive statistics</description>
+      <test-commands>
+        <cache-admin-command>-addPool pool1</cache-admin-command>
+        <cache-admin-command>-addDirective -path /foo -pool pool1 -ttl 2d</cache-admin-command>
+        <cache-admin-command>-addDirective -path /bar -pool pool1 -ttl 24h</cache-admin-command>
+        <cache-admin-command>-addDirective -path /baz -replication 2 -pool pool1 -ttl 60m</cache-admin-command>
+        <cache-admin-command>-listDirectives -pool pool1 -stats</cache-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <cache-admin-command>-removePool pool1</cache-admin-command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Found 3 entries</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>/foo              0             0             0             0</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>/bar              0             0             0             0</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>/baz              0             0             0             0</expected-output>
+        </comparator>
+      </comparators>
+    </test>
   </tests>
 </configuration>

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

@@ -229,6 +229,8 @@ Release 2.4.0 - UNRELEASED
     MAPREDUCE-5645. TestFixedLengthInputFormat fails with native libs (Mit
     Desai via jeagles)
 
+    MAPREDUCE-5632. TestRMContainerAllocator#testUpdatedNodes fails (jeagles)
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -278,6 +280,10 @@ Release 2.3.0 - UNRELEASED
     MAPREDUCE-5451. MR uses LD_LIBRARY_PATH which doesn't mean anything in
     Windows. (Yingda Chen via cnauroth)
 
+    MAPREDUCE-5409. MRAppMaster throws InvalidStateTransitonException: Invalid
+    event: TA_TOO_MANY_FETCH_FAILURE at KILLED for TaskAttemptImpl (Gera
+    Shegalov via jlowe)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

+ 17 - 23
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java

@@ -192,6 +192,21 @@ public abstract class TaskAttemptImpl implements
     DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION 
       = new DiagnosticInformationUpdater();
 
+  private static final EnumSet<TaskAttemptEventType>
+    FAILED_KILLED_STATE_IGNORED_EVENTS = EnumSet.of(
+      TaskAttemptEventType.TA_KILL,
+      TaskAttemptEventType.TA_ASSIGNED,
+      TaskAttemptEventType.TA_CONTAINER_COMPLETED,
+      TaskAttemptEventType.TA_UPDATE,
+      // Container launch events can arrive late
+      TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
+      TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED,
+      TaskAttemptEventType.TA_CONTAINER_CLEANED,
+      TaskAttemptEventType.TA_COMMIT_PENDING,
+      TaskAttemptEventType.TA_DONE,
+      TaskAttemptEventType.TA_FAILMSG,
+      TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURE);
+
   private static final StateMachineFactory
         <TaskAttemptImpl, TaskAttemptStateInternal, TaskAttemptEventType, TaskAttemptEvent>
         stateMachineFactory
@@ -452,18 +467,7 @@ public abstract class TaskAttemptImpl implements
        DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
      // Ignore-able events for FAILED state
      .addTransition(TaskAttemptStateInternal.FAILED, TaskAttemptStateInternal.FAILED,
-         EnumSet.of(TaskAttemptEventType.TA_KILL,
-             TaskAttemptEventType.TA_ASSIGNED,
-             TaskAttemptEventType.TA_CONTAINER_COMPLETED,
-             TaskAttemptEventType.TA_UPDATE,
-             // Container launch events can arrive late
-             TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
-             TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED,
-             TaskAttemptEventType.TA_CONTAINER_CLEANED,
-             TaskAttemptEventType.TA_COMMIT_PENDING,
-             TaskAttemptEventType.TA_DONE,
-             TaskAttemptEventType.TA_FAILMSG,
-             TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURE))
+       FAILED_KILLED_STATE_IGNORED_EVENTS)
 
      // Transitions from KILLED state
      .addTransition(TaskAttemptStateInternal.KILLED, TaskAttemptStateInternal.KILLED,
@@ -471,17 +475,7 @@ public abstract class TaskAttemptImpl implements
          DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
      // Ignore-able events for KILLED state
      .addTransition(TaskAttemptStateInternal.KILLED, TaskAttemptStateInternal.KILLED,
-         EnumSet.of(TaskAttemptEventType.TA_KILL,
-             TaskAttemptEventType.TA_ASSIGNED,
-             TaskAttemptEventType.TA_CONTAINER_COMPLETED,
-             TaskAttemptEventType.TA_UPDATE,
-             // Container launch events can arrive late
-             TaskAttemptEventType.TA_CONTAINER_LAUNCHED,
-             TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED,
-             TaskAttemptEventType.TA_CONTAINER_CLEANED,
-             TaskAttemptEventType.TA_COMMIT_PENDING,
-             TaskAttemptEventType.TA_DONE,
-             TaskAttemptEventType.TA_FAILMSG))
+       FAILED_KILLED_STATE_IGNORED_EVENTS)
 
      // create the topology tables
      .installTopology();

+ 12 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java

@@ -101,6 +101,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoSchedule
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 @SuppressWarnings("unchecked")
@@ -111,6 +112,12 @@ public class TestRMContainerAllocator {
   static final RecordFactory recordFactory = RecordFactoryProvider
       .getRecordFactory(null);
 
+  @Before
+  public void setup() {
+    MyContainerAllocator.getJobUpdatedNodeEvents().clear();
+    MyContainerAllocator.getTaskAttemptKillEvents().clear();
+  }
+
   @After
   public void tearDown() {
     DefaultMetricsSystem.shutdown();
@@ -770,6 +777,9 @@ public class TestRMContainerAllocator {
 
     nm1.nodeHeartbeat(true);
     dispatcher.await();
+    Assert.assertEquals(1, allocator.getJobUpdatedNodeEvents().size());
+    Assert.assertEquals(3, allocator.getJobUpdatedNodeEvents().get(0).getUpdatedNodes().size());
+    allocator.getJobUpdatedNodeEvents().clear();
     // get the assignment
     assigned = allocator.schedule();
     dispatcher.await();
@@ -1501,11 +1511,11 @@ public class TestRMContainerAllocator {
       return result;
     }
     
-    List<TaskAttemptKillEvent> getTaskAttemptKillEvents() {
+    static List<TaskAttemptKillEvent> getTaskAttemptKillEvents() {
       return taskAttemptKillEvents;
     }
     
-    List<JobUpdatedNodesEvent> getJobUpdatedNodeEvents() {
+    static List<JobUpdatedNodesEvent> getJobUpdatedNodeEvents() {
       return jobUpdatedNodeEvents;
     }
 

+ 68 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java

@@ -550,6 +550,8 @@ public class TestTaskAttempt{
         eventHandler.internalError);
   }
 
+
+
   @Test
   public void testAppDiognosticEventOnUnassignedTask() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(1, 2);
@@ -599,6 +601,72 @@ public class TestTaskAttempt{
         eventHandler.internalError);
   }
 
+  @Test
+  public void testTooManyFetchFailureAfterKill() throws Exception {
+    ApplicationId appId = ApplicationId.newInstance(1, 2);
+    ApplicationAttemptId appAttemptId =
+      ApplicationAttemptId.newInstance(appId, 0);
+    JobId jobId = MRBuilderUtils.newJobId(appId, 1);
+    TaskId taskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.MAP);
+    TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId, 0);
+    Path jobFile = mock(Path.class);
+
+    MockEventHandler eventHandler = new MockEventHandler();
+    TaskAttemptListener taListener = mock(TaskAttemptListener.class);
+    when(taListener.getAddress()).thenReturn(new InetSocketAddress("localhost", 0));
+
+    JobConf jobConf = new JobConf();
+    jobConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
+    jobConf.setBoolean("fs.file.impl.disable.cache", true);
+    jobConf.set(JobConf.MAPRED_MAP_TASK_ENV, "");
+    jobConf.set(MRJobConfig.APPLICATION_ATTEMPT_ID, "10");
+
+    TaskSplitMetaInfo splits = mock(TaskSplitMetaInfo.class);
+    when(splits.getLocations()).thenReturn(new String[] {"127.0.0.1"});
+
+    AppContext appCtx = mock(AppContext.class);
+    ClusterInfo clusterInfo = mock(ClusterInfo.class);
+    Resource resource = mock(Resource.class);
+    when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
+    when(resource.getMemory()).thenReturn(1024);
+
+    TaskAttemptImpl taImpl =
+      new MapTaskAttemptImpl(taskId, 1, eventHandler, jobFile, 1,
+        splits, jobConf, taListener,
+        mock(Token.class), new Credentials(),
+        new SystemClock(), appCtx);
+
+    NodeId nid = NodeId.newInstance("127.0.0.1", 0);
+    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
+    Container container = mock(Container.class);
+    when(container.getId()).thenReturn(contId);
+    when(container.getNodeId()).thenReturn(nid);
+    when(container.getNodeHttpAddress()).thenReturn("localhost:0");
+
+    taImpl.handle(new TaskAttemptEvent(attemptId,
+      TaskAttemptEventType.TA_SCHEDULE));
+    taImpl.handle(new TaskAttemptContainerAssignedEvent(attemptId,
+      container, mock(Map.class)));
+    taImpl.handle(new TaskAttemptContainerLaunchedEvent(attemptId, 0));
+    taImpl.handle(new TaskAttemptEvent(attemptId,
+      TaskAttemptEventType.TA_DONE));
+    taImpl.handle(new TaskAttemptEvent(attemptId,
+      TaskAttemptEventType.TA_CONTAINER_CLEANED));
+
+    assertEquals("Task attempt is not in succeeded state", taImpl.getState(),
+      TaskAttemptState.SUCCEEDED);
+    taImpl.handle(new TaskAttemptEvent(attemptId,
+      TaskAttemptEventType.TA_KILL));
+    assertEquals("Task attempt is not in KILLED state", taImpl.getState(),
+      TaskAttemptState.KILLED);
+    taImpl.handle(new TaskAttemptEvent(attemptId,
+      TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURE));
+    assertEquals("Task attempt is not in KILLED state, still", taImpl.getState(),
+      TaskAttemptState.KILLED);
+    assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED",
+      eventHandler.internalError);
+  }
+
   @Test
   public void testAppDiognosticEventOnNewTask() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(1, 2);

+ 1 - 1
hadoop-project/pom.xml

@@ -467,7 +467,7 @@
       <dependency>
         <groupId>commons-io</groupId>
         <artifactId>commons-io</artifactId>
-        <version>2.1</version>
+        <version>2.4</version>
       </dependency>
 
       <dependency>

+ 5 - 0
hadoop-tools/hadoop-distcp/pom.xml

@@ -95,6 +95,11 @@
       <scope>test</scope>
       <type>test-jar</type>
     </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

+ 11 - 6
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/SimpleCopyListing.java

@@ -32,6 +32,8 @@ import org.apache.hadoop.tools.util.DistCpUtils;
 import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.security.Credentials;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import java.io.*;
 import java.util.Stack;
 
@@ -107,12 +109,13 @@ public class SimpleCopyListing extends CopyListing {
   /** {@inheritDoc} */
   @Override
   public void doBuildListing(Path pathToListingFile, DistCpOptions options) throws IOException {
-
-    SequenceFile.Writer fileListWriter = null;
-
+    doBuildListing(getWriter(pathToListingFile), options);
+  }
+  
+  @VisibleForTesting
+  public void doBuildListing(SequenceFile.Writer fileListWriter,
+      DistCpOptions options) throws IOException {
     try {
-      fileListWriter = getWriter(pathToListingFile);
-
       for (Path path: options.getSourcePaths()) {
         FileSystem sourceFS = path.getFileSystem(getConf());
         path = makeQualified(path);
@@ -143,8 +146,10 @@ public class SimpleCopyListing extends CopyListing {
               localFile, options);
         }
       }
+      fileListWriter.close();
+      fileListWriter = null;
     } finally {
-      IOUtils.closeStream(fileListWriter);
+      IOUtils.cleanup(LOG, fileListWriter);
     }
   }
 

+ 6 - 1
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java

@@ -30,6 +30,8 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import java.io.*;
 import java.util.EnumSet;
 
@@ -176,7 +178,8 @@ public class RetriableFileCopyCommand extends RetriableCommand {
     return new Path(root, ".distcp.tmp." + context.getTaskAttemptID().toString());
   }
 
-  private long copyBytes(FileStatus sourceFileStatus, OutputStream outStream,
+  @VisibleForTesting
+  long copyBytes(FileStatus sourceFileStatus, OutputStream outStream,
                          int bufferSize, Mapper.Context context)
       throws IOException {
     Path source = sourceFileStatus.getPath();
@@ -193,6 +196,8 @@ public class RetriableFileCopyCommand extends RetriableCommand {
         updateContextStatus(totalBytesRead, context, sourceFileStatus);
         bytesRead = inStream.read(buf);
       }
+      outStream.close();
+      outStream = null;
     } finally {
       IOUtils.cleanup(LOG, outStream, inStream);
     }

+ 28 - 0
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestCopyListing.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.tools;
 
+import static org.mockito.Mockito.*;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
@@ -36,6 +38,7 @@ import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.AfterClass;
 
+import java.io.File;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.List;
@@ -282,4 +285,29 @@ public class TestCopyListing extends SimpleCopyListing {
       IOUtils.closeStream(reader);
     }
   }
+  
+  @Test
+  public void testFailOnCloseError() throws IOException {
+    File inFile = File.createTempFile("TestCopyListingIn", null);
+    inFile.deleteOnExit();
+    File outFile = File.createTempFile("TestCopyListingOut", null);
+    outFile.deleteOnExit();
+    List<Path> srcs = new ArrayList<Path>();
+    srcs.add(new Path(inFile.toURI()));
+    
+    Exception expectedEx = new IOException("boom");
+    SequenceFile.Writer writer = mock(SequenceFile.Writer.class);
+    doThrow(expectedEx).when(writer).close();
+    
+    SimpleCopyListing listing = new SimpleCopyListing(getConf(), CREDENTIALS);
+    DistCpOptions options = new DistCpOptions(srcs, new Path(outFile.toURI()));
+    Exception actualEx = null;
+    try {
+      listing.doBuildListing(writer, options);
+    } catch (Exception e) {
+      actualEx = e;
+    }
+    Assert.assertNotNull("close writer didn't fail", actualEx);
+    Assert.assertEquals(expectedEx, actualEx);
+  }
 }

+ 59 - 0
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestRetriableFileCopyCommand.java

@@ -0,0 +1,59 @@
+/**
+ * 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.tools.mapred;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.*;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+
+public class TestRetriableFileCopyCommand {
+  @SuppressWarnings("rawtypes")
+  @Test
+  public void testFailOnCloseError() throws Exception {
+    Mapper.Context context = mock(Mapper.Context.class);
+    doReturn(new Configuration()).when(context).getConfiguration();
+
+    Exception expectedEx = new IOException("boom");
+    OutputStream out = mock(OutputStream.class);
+    doThrow(expectedEx).when(out).close();
+
+    File f = File.createTempFile(this.getClass().getSimpleName(), null);
+    f.deleteOnExit();
+    FileStatus stat =
+        new FileStatus(1L, false, 1, 1024, 0, new Path(f.toURI()));
+    
+    Exception actualEx = null;
+    try {
+      new RetriableFileCopyCommand("testFailOnCloseError")
+        .copyBytes(stat, out, 512, context);
+    } catch (Exception e) {
+      actualEx = e;
+    }
+    assertNotNull("close didn't fail", actualEx);
+    assertEquals(expectedEx, actualEx);
+  }  
+}

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

@@ -43,6 +43,9 @@ Release 2.4.0 - UNRELEASED
     YARN-1392. Allow sophisticated app-to-queue placement policies in the Fair
     Scheduler (Sandy Ryza)
 
+    YARN-1447. Common PB type definitions for container resizing. (Wangda Tan
+    via Sandy Ryza)
+
   IMPROVEMENTS
 
     YARN-7. Support CPU resource for DistributedShell. (Junping Du via llu)
@@ -135,6 +138,9 @@ Release 2.4.0 - UNRELEASED
     YARN-1332. In TestAMRMClient, replace assertTrue with assertEquals where
     possible (Sebastian Wong via Sandy Ryza)
 
+    YARN-1403. Separate out configuration loading from QueueManager in the Fair
+    Scheduler (Sandy Ryza)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 78 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceDecrease.java

@@ -0,0 +1,78 @@
+/**
+ * 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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Used by Application Master to ask Node Manager reduce size of a specified
+ * container
+ */
+public abstract class ContainerResourceDecrease {
+  @Public
+  public static ContainerResourceDecrease newInstance(
+      ContainerId existingContainerId, Resource targetCapability) {
+    ContainerResourceDecrease context = Records
+        .newRecord(ContainerResourceDecrease.class);
+    context.setContainerId(existingContainerId);
+    context.setCapability(targetCapability);
+    return context;
+  }
+
+  @Public
+  public abstract ContainerId getContainerId();
+
+  @Public
+  public abstract void setContainerId(ContainerId containerId);
+
+  @Public
+  public abstract Resource getCapability();
+
+  @Public
+  public abstract void setCapability(Resource capability);
+  
+  @Override
+  public int hashCode() {
+    return getCapability().hashCode() + getContainerId().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other instanceof ContainerResourceDecrease) {
+      ContainerResourceDecrease ctx = (ContainerResourceDecrease)other;
+      
+      if (getContainerId() == null && ctx.getContainerId() != null) {
+        return false;
+      } else if (!getContainerId().equals(ctx.getContainerId())) {
+        return false;
+      }
+      
+      if (getCapability() == null && ctx.getCapability() != null) {
+        return false;
+      } else if (!getCapability().equals(ctx.getCapability())) {
+        return false;
+      }
+      
+      return true;
+    } else {
+      return false;
+    }
+  }
+}

+ 84 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncrease.java

@@ -0,0 +1,84 @@
+/**
+ * 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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Represent a new increased container accepted by Resource Manager
+ */
+public abstract class ContainerResourceIncrease {
+  @Public
+  public static ContainerResourceIncrease newInstance(
+      ContainerId existingContainerId, Resource targetCapability, Token token) {
+    ContainerResourceIncrease context = Records
+        .newRecord(ContainerResourceIncrease.class);
+    context.setContainerId(existingContainerId);
+    context.setCapability(targetCapability);
+    context.setContainerToken(token);
+    return context;
+  }
+
+  @Public
+  public abstract ContainerId getContainerId();
+
+  @Public
+  public abstract void setContainerId(ContainerId containerId);
+
+  @Public
+  public abstract Resource getCapability();
+
+  @Public
+  public abstract void setCapability(Resource capability);
+  
+  @Public
+  public abstract Token getContainerToken();
+
+  @Public
+  public abstract void setContainerToken(Token token);
+
+  @Override
+  public int hashCode() {
+    return getCapability().hashCode() + getContainerId().hashCode();
+  }
+  
+  @Override
+  public boolean equals(Object other) {
+    if (other instanceof ContainerResourceIncrease) {
+      ContainerResourceIncrease ctx = (ContainerResourceIncrease)other;
+      
+      if (getContainerId() == null && ctx.getContainerId() != null) {
+        return false;
+      } else if (!getContainerId().equals(ctx.getContainerId())) {
+        return false;
+      }
+      
+      if (getCapability() == null && ctx.getCapability() != null) {
+        return false;
+      } else if (!getCapability().equals(ctx.getCapability())) {
+        return false;
+      }
+      
+      return true;
+    } else {
+      return false;
+    }
+  }
+}

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncreaseRequest.java

@@ -0,0 +1,80 @@
+/**
+ * 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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Used by Application Master, send a container resource increase request to
+ * Resource Manager
+ */
+@Public
+public abstract class ContainerResourceIncreaseRequest {
+  @Public
+  public static ContainerResourceIncreaseRequest newInstance(
+      ContainerId existingContainerId, Resource targetCapability) {
+    ContainerResourceIncreaseRequest context = Records
+        .newRecord(ContainerResourceIncreaseRequest.class);
+    context.setContainerId(existingContainerId);
+    context.setCapability(targetCapability);
+    return context;
+  }
+
+  @Public
+  public abstract ContainerId getContainerId();
+
+  @Public
+  public abstract void setContainerId(ContainerId containerId);
+
+  @Public
+  public abstract Resource getCapability();
+
+  @Public
+  public abstract void setCapability(Resource capability);
+
+  @Override
+  public int hashCode() {
+    return getCapability().hashCode() + getContainerId().hashCode();
+  }
+  
+  @Override
+  public boolean equals(Object other) {
+    if (other instanceof ContainerResourceIncreaseRequest) {
+      ContainerResourceIncreaseRequest ctx =
+          (ContainerResourceIncreaseRequest) other;
+      
+      if (getContainerId() == null && ctx.getContainerId() != null) {
+        return false;
+      } else if (!getContainerId().equals(ctx.getContainerId())) {
+        return false;
+      }
+      
+      if (getCapability() == null && ctx.getCapability() != null) {
+        return false;
+      } else if (!getCapability().equals(ctx.getCapability())) {
+        return false;
+      }
+      
+      return true;
+    } else {
+      return false;
+    }
+  }
+}

+ 16 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto

@@ -311,6 +311,22 @@ enum ContainerExitStatusProto {
   DISKS_FAILED = -101;
 }
 
+message ContainerResourceIncreaseRequestProto {
+  optional ContainerIdProto container_id = 1;
+  optional ResourceProto capability = 2;
+} 
+
+message ContainerResourceIncreaseProto {
+  optional ContainerIdProto container_id = 1;
+  optional ResourceProto capability = 2;
+  optional hadoop.common.TokenProto container_token = 3;
+}
+
+message ContainerResourceDecreaseProto {
+  optional ContainerIdProto container_id = 1;
+  optional ResourceProto capability = 2;
+}
+
 ////////////////////////////////////////////////////////////////////////
 ////// From common//////////////////////////////////////////////////////
 ////////////////////////////////////////////////////////////////////////

+ 136 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceDecreasePBImpl.java

@@ -0,0 +1,136 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
+
+public class ContainerResourceDecreasePBImpl extends ContainerResourceDecrease {
+  ContainerResourceDecreaseProto proto = ContainerResourceDecreaseProto
+      .getDefaultInstance();
+  ContainerResourceDecreaseProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ContainerId existingContainerId = null;
+  private Resource targetCapability = null;
+
+  public ContainerResourceDecreasePBImpl() {
+    builder = ContainerResourceDecreaseProto.newBuilder();
+  }
+
+  public ContainerResourceDecreasePBImpl(ContainerResourceDecreaseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ContainerResourceDecreaseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public ContainerId getContainerId() {
+    ContainerResourceDecreaseProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.existingContainerId != null) {
+      return this.existingContainerId;
+    }
+    if (p.hasContainerId()) {
+      this.existingContainerId = convertFromProtoFormat(p.getContainerId());
+    }
+    return this.existingContainerId;
+  }
+
+  @Override
+  public void setContainerId(ContainerId existingContainerId) {
+    maybeInitBuilder();
+    if (existingContainerId == null) {
+      builder.clearContainerId();
+    }
+    this.existingContainerId = existingContainerId;
+  }
+
+  @Override
+  public Resource getCapability() {
+    ContainerResourceDecreaseProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.targetCapability != null) {
+      return this.targetCapability;
+    }
+    if (p.hasCapability()) {
+      this.targetCapability = convertFromProtoFormat(p.getCapability());
+    }
+    return this.targetCapability;
+  }
+
+  @Override
+  public void setCapability(Resource targetCapability) {
+    maybeInitBuilder();
+    if (targetCapability == null) {
+      builder.clearCapability();
+    }
+    this.targetCapability = targetCapability;
+  }
+
+  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
+    return new ContainerIdPBImpl(p);
+  }
+
+  private ContainerIdProto convertToProtoFormat(ContainerId t) {
+    return ((ContainerIdPBImpl) t).getProto();
+  }
+
+  private Resource convertFromProtoFormat(ResourceProto p) {
+    return new ResourcePBImpl(p);
+  }
+
+  private ResourceProto convertToProtoFormat(Resource t) {
+    return ((ResourcePBImpl) t).getProto();
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ContainerResourceDecreaseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.existingContainerId != null) {
+      builder.setContainerId(convertToProtoFormat(this.existingContainerId));
+    }
+    if (this.targetCapability != null) {
+      builder.setCapability(convertToProtoFormat(this.targetCapability));
+    }
+  }
+}

+ 171 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreasePBImpl.java

@@ -0,0 +1,171 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
+
+public class ContainerResourceIncreasePBImpl extends ContainerResourceIncrease {
+  ContainerResourceIncreaseProto proto = ContainerResourceIncreaseProto
+      .getDefaultInstance();
+  ContainerResourceIncreaseProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ContainerId existingContainerId = null;
+  private Resource targetCapability = null;
+  private Token token = null;
+
+  public ContainerResourceIncreasePBImpl() {
+    builder = ContainerResourceIncreaseProto.newBuilder();
+  }
+
+  public ContainerResourceIncreasePBImpl(ContainerResourceIncreaseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ContainerResourceIncreaseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public ContainerId getContainerId() {
+    ContainerResourceIncreaseProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.existingContainerId != null) {
+      return this.existingContainerId;
+    }
+    if (p.hasContainerId()) {
+      this.existingContainerId = convertFromProtoFormat(p.getContainerId());
+    }
+    return this.existingContainerId;
+  }
+
+  @Override
+  public void setContainerId(ContainerId existingContainerId) {
+    maybeInitBuilder();
+    if (existingContainerId == null) {
+      builder.clearContainerId();
+    }
+    this.existingContainerId = existingContainerId;
+  }
+
+  @Override
+  public Resource getCapability() {
+    ContainerResourceIncreaseProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.targetCapability != null) {
+      return this.targetCapability;
+    }
+    if (p.hasCapability()) {
+      this.targetCapability = convertFromProtoFormat(p.getCapability());
+    }
+    return this.targetCapability;
+  }
+
+  @Override
+  public void setCapability(Resource targetCapability) {
+    maybeInitBuilder();
+    if (targetCapability == null) {
+      builder.clearCapability();
+    }
+    this.targetCapability = targetCapability;
+  }
+  
+  @Override
+  public Token getContainerToken() {
+    ContainerResourceIncreaseProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.token != null) {
+      return this.token;
+    }
+    if (p.hasContainerToken()) {
+      this.token = convertFromProtoFormat(p.getContainerToken());
+    }
+    return this.token;
+  }
+
+  @Override
+  public void setContainerToken(Token token) {
+    maybeInitBuilder();
+    if (token == null) {
+      builder.clearContainerToken();
+    }
+    this.token = token;
+  }
+
+  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
+    return new ContainerIdPBImpl(p);
+  }
+
+  private ContainerIdProto convertToProtoFormat(ContainerId t) {
+    return ((ContainerIdPBImpl) t).getProto();
+  }
+
+  private Resource convertFromProtoFormat(ResourceProto p) {
+    return new ResourcePBImpl(p);
+  }
+
+  private ResourceProto convertToProtoFormat(Resource t) {
+    return ((ResourcePBImpl) t).getProto();
+  }
+  
+  private Token convertFromProtoFormat(TokenProto p) {
+    return new TokenPBImpl(p);
+  }
+
+  private TokenProto convertToProtoFormat(Token t) {
+    return ((TokenPBImpl) t).getProto();
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ContainerResourceIncreaseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.existingContainerId != null) {
+      builder.setContainerId(convertToProtoFormat(this.existingContainerId));
+    }
+    if (this.targetCapability != null) {
+      builder.setCapability(convertToProtoFormat(this.targetCapability));
+    }
+    if (this.token != null) {
+      builder.setContainerToken(convertToProtoFormat(this.token));
+    }
+  }
+}

+ 141 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreaseRequestPBImpl.java

@@ -0,0 +1,141 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
+
+
+public class ContainerResourceIncreaseRequestPBImpl extends
+    ContainerResourceIncreaseRequest {
+  ContainerResourceIncreaseRequestProto proto = 
+      ContainerResourceIncreaseRequestProto.getDefaultInstance();
+  ContainerResourceIncreaseRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ContainerId existingContainerId = null;
+  private Resource targetCapability = null;
+
+  public ContainerResourceIncreaseRequestPBImpl() {
+    builder = ContainerResourceIncreaseRequestProto.newBuilder();
+  }
+
+  public ContainerResourceIncreaseRequestPBImpl(
+      ContainerResourceIncreaseRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ContainerResourceIncreaseRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public ContainerId getContainerId() {
+    ContainerResourceIncreaseRequestProtoOrBuilder p = viaProto ? proto
+        : builder;
+    if (this.existingContainerId != null) {
+      return this.existingContainerId;
+    }
+    if (p.hasContainerId()) {
+      this.existingContainerId = convertFromProtoFormat(p.getContainerId());
+    }
+    return this.existingContainerId;
+  }
+
+  @Override
+  public void setContainerId(ContainerId existingContainerId) {
+    maybeInitBuilder();
+    if (existingContainerId == null) {
+      builder.clearContainerId();
+    }
+    this.existingContainerId = existingContainerId;
+  }
+
+  @Override
+  public Resource getCapability() {
+    ContainerResourceIncreaseRequestProtoOrBuilder p = viaProto ? proto
+        : builder;
+    if (this.targetCapability != null) {
+      return this.targetCapability;
+    }
+    if (p.hasCapability()) {
+      this.targetCapability = convertFromProtoFormat(p.getCapability());
+    }
+    return this.targetCapability;
+  }
+
+  @Override
+  public void setCapability(Resource targetCapability) {
+    maybeInitBuilder();
+    if (targetCapability == null) {
+      builder.clearCapability();
+    }
+    this.targetCapability = targetCapability;
+  }
+
+  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
+    return new ContainerIdPBImpl(p);
+  }
+
+  private ContainerIdProto convertToProtoFormat(ContainerId t) {
+    return ((ContainerIdPBImpl) t).getProto();
+  }
+
+  private Resource convertFromProtoFormat(ResourceProto p) {
+    return new ResourcePBImpl(p);
+  }
+
+  private ResourceProto convertToProtoFormat(Resource t) {
+    return ((ResourcePBImpl) t).getProto();
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ContainerResourceIncreaseRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.existingContainerId != null) {
+      builder.setContainerId(convertToProtoFormat(this.existingContainerId));
+    }
+    if (this.targetCapability != null) {
+      builder.setCapability(convertToProtoFormat(this.targetCapability));
+    }
+  }
+}

+ 66 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceDecrease.java

@@ -0,0 +1,66 @@
+/**
+ * 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.api;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceDecreasePBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProto;
+import org.junit.Test;
+
+public class TestContainerResourceDecrease {
+  @Test
+  public void testResourceDecreaseContext() {
+    ContainerId containerId = ContainerId
+        .newInstance(ApplicationAttemptId.newInstance(
+            ApplicationId.newInstance(1234, 3), 3), 7);
+    Resource resource = Resource.newInstance(1023, 3);
+    ContainerResourceDecrease ctx = ContainerResourceDecrease.newInstance(
+        containerId, resource);
+
+    // get proto and recover to ctx
+    ContainerResourceDecreaseProto proto = 
+        ((ContainerResourceDecreasePBImpl) ctx).getProto();
+    ctx = new ContainerResourceDecreasePBImpl(proto);
+
+    // check values
+    Assert.assertEquals(ctx.getCapability(), resource);
+    Assert.assertEquals(ctx.getContainerId(), containerId);
+  }
+  
+  @Test
+  public void testResourceDecreaseContextWithNull() {
+    ContainerResourceDecrease ctx = ContainerResourceDecrease.newInstance(null,
+        null);
+    
+    // get proto and recover to ctx;
+    ContainerResourceDecreaseProto proto = 
+        ((ContainerResourceDecreasePBImpl) ctx).getProto();
+    ctx = new ContainerResourceDecreasePBImpl(proto);
+
+    // check values
+    Assert.assertNull(ctx.getCapability());
+    Assert.assertNull(ctx.getContainerId());
+  }
+}

+ 74 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncrease.java

@@ -0,0 +1,74 @@
+/**
+ * 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.api;
+
+import java.util.Arrays;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreasePBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProto;
+import org.junit.Test;
+
+public class TestContainerResourceIncrease {
+  @Test
+  public void testResourceIncreaseContext() {
+    byte[] identifier = new byte[] { 1, 2, 3, 4 };
+    Token token = Token.newInstance(identifier, "", "".getBytes(), "");
+    ContainerId containerId = ContainerId
+        .newInstance(ApplicationAttemptId.newInstance(
+            ApplicationId.newInstance(1234, 3), 3), 7);
+    Resource resource = Resource.newInstance(1023, 3);
+    ContainerResourceIncrease ctx = ContainerResourceIncrease.newInstance(
+        containerId, resource, token);
+
+    // get proto and recover to ctx
+    ContainerResourceIncreaseProto proto = 
+        ((ContainerResourceIncreasePBImpl) ctx).getProto();
+    ctx = new ContainerResourceIncreasePBImpl(proto);
+
+    // check values
+    Assert.assertEquals(ctx.getCapability(), resource);
+    Assert.assertEquals(ctx.getContainerId(), containerId);
+    Assert.assertTrue(Arrays.equals(ctx.getContainerToken().getIdentifier()
+        .array(), identifier));
+  }
+  
+  @Test
+  public void testResourceIncreaseContextWithNull() {
+    ContainerResourceIncrease ctx = ContainerResourceIncrease.newInstance(null,
+        null, null);
+    
+    // get proto and recover to ctx;
+    ContainerResourceIncreaseProto proto = 
+        ((ContainerResourceIncreasePBImpl) ctx).getProto();
+    ctx = new ContainerResourceIncreasePBImpl(proto);
+
+    // check values
+    Assert.assertNull(ctx.getContainerToken());
+    Assert.assertNull(ctx.getCapability());
+    Assert.assertNull(ctx.getContainerId());
+  }
+}

+ 68 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncreaseRequest.java

@@ -0,0 +1,68 @@
+/**
+ * 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.api;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreaseRequestPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProto;
+import org.junit.Test;
+
+public class TestContainerResourceIncreaseRequest {
+  @Test
+  public void ContainerResourceIncreaseRequest() {
+    ContainerId containerId = ContainerId
+        .newInstance(ApplicationAttemptId.newInstance(
+            ApplicationId.newInstance(1234, 3), 3), 7);
+    Resource resource = Resource.newInstance(1023, 3);
+    ContainerResourceIncreaseRequest context = ContainerResourceIncreaseRequest
+        .newInstance(containerId, resource);
+
+    // to proto and get it back
+    ContainerResourceIncreaseRequestProto proto = 
+        ((ContainerResourceIncreaseRequestPBImpl) context).getProto();
+    ContainerResourceIncreaseRequest contextRecover = 
+        new ContainerResourceIncreaseRequestPBImpl(proto);
+
+    // check value
+    Assert.assertEquals(contextRecover.getContainerId(), containerId);
+    Assert.assertEquals(contextRecover.getCapability(), resource);
+  }
+
+  @Test
+  public void testResourceChangeContextWithNullField() {
+    ContainerResourceIncreaseRequest context = ContainerResourceIncreaseRequest
+        .newInstance(null, null);
+
+    // to proto and get it back
+    ContainerResourceIncreaseRequestProto proto = 
+        ((ContainerResourceIncreaseRequestPBImpl) context).getProto();
+    ContainerResourceIncreaseRequest contextRecover = 
+        new ContainerResourceIncreaseRequestPBImpl(proto);
+
+    // check value
+    Assert.assertNull(contextRecover.getContainerId());
+    Assert.assertNull(contextRecover.getCapability());
+  }
+}

+ 229 - 0
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

@@ -0,0 +1,229 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.yarn.api.records.QueueACL;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public class AllocationConfiguration {
+  private static final AccessControlList EVERYBODY_ACL = new AccessControlList("*");
+  private static final AccessControlList NOBODY_ACL = new AccessControlList(" ");
+  
+  // Minimum resource allocation for each queue
+  private final Map<String, Resource> minQueueResources;
+  // Maximum amount of resources per queue
+  private final Map<String, Resource> maxQueueResources;
+  // Sharing weights for each queue
+  private final Map<String, ResourceWeights> queueWeights;
+  
+  // Max concurrent running applications for each queue and for each user; in addition,
+  // for users that have no max specified, we use the userMaxJobsDefault.
+  @VisibleForTesting
+  final Map<String, Integer> queueMaxApps;
+  @VisibleForTesting
+  final Map<String, Integer> userMaxApps;
+  private final int userMaxAppsDefault;
+  private final int queueMaxAppsDefault;
+
+  // ACL's for each queue. Only specifies non-default ACL's from configuration.
+  private final Map<String, Map<QueueACL, AccessControlList>> queueAcls;
+
+  // Min share preemption timeout for each queue in seconds. If a job in the queue
+  // waits this long without receiving its guaranteed share, it is allowed to
+  // preempt other jobs' tasks.
+  private final Map<String, Long> minSharePreemptionTimeouts;
+
+  // Default min share preemption timeout for queues where it is not set
+  // explicitly.
+  private final long defaultMinSharePreemptionTimeout;
+
+  // Preemption timeout for jobs below fair share in seconds. If a job remains
+  // below half its fair share for this long, it is allowed to preempt tasks.
+  private final long fairSharePreemptionTimeout;
+
+  private final Map<String, SchedulingPolicy> schedulingPolicies;
+  
+  private final SchedulingPolicy defaultSchedulingPolicy;
+  
+  // Policy for mapping apps to queues
+  @VisibleForTesting
+  QueuePlacementPolicy placementPolicy;
+  
+  private final Set<String> queueNames;
+  
+  public AllocationConfiguration(Map<String, Resource> minQueueResources, 
+      Map<String, Resource> maxQueueResources, 
+      Map<String, Integer> queueMaxApps, Map<String, Integer> userMaxApps,
+      Map<String, ResourceWeights> queueWeights, int userMaxAppsDefault,
+      int queueMaxAppsDefault, Map<String, SchedulingPolicy> schedulingPolicies,
+      SchedulingPolicy defaultSchedulingPolicy,
+      Map<String, Long> minSharePreemptionTimeouts, 
+      Map<String, Map<QueueACL, AccessControlList>> queueAcls,
+      long fairSharePreemptionTimeout, long defaultMinSharePreemptionTimeout,
+      QueuePlacementPolicy placementPolicy, Set<String> queueNames) {
+    this.minQueueResources = minQueueResources;
+    this.maxQueueResources = maxQueueResources;
+    this.queueMaxApps = queueMaxApps;
+    this.userMaxApps = userMaxApps;
+    this.queueWeights = queueWeights;
+    this.userMaxAppsDefault = userMaxAppsDefault;
+    this.queueMaxAppsDefault = queueMaxAppsDefault;
+    this.defaultSchedulingPolicy = defaultSchedulingPolicy;
+    this.schedulingPolicies = schedulingPolicies;
+    this.minSharePreemptionTimeouts = minSharePreemptionTimeouts;
+    this.queueAcls = queueAcls;
+    this.fairSharePreemptionTimeout = fairSharePreemptionTimeout;
+    this.defaultMinSharePreemptionTimeout = defaultMinSharePreemptionTimeout;
+    this.placementPolicy = placementPolicy;
+    this.queueNames = queueNames;
+  }
+  
+  public AllocationConfiguration(Configuration conf) {
+    minQueueResources = new HashMap<String, Resource>();
+    maxQueueResources = new HashMap<String, Resource>();
+    queueWeights = new HashMap<String, ResourceWeights>();
+    queueMaxApps = new HashMap<String, Integer>();
+    userMaxApps = new HashMap<String, Integer>();
+    userMaxAppsDefault = Integer.MAX_VALUE;
+    queueMaxAppsDefault = Integer.MAX_VALUE;
+    queueAcls = new HashMap<String, Map<QueueACL, AccessControlList>>();
+    minSharePreemptionTimeouts = new HashMap<String, Long>();
+    defaultMinSharePreemptionTimeout = Long.MAX_VALUE;
+    fairSharePreemptionTimeout = Long.MAX_VALUE;
+    schedulingPolicies = new HashMap<String, SchedulingPolicy>();
+    defaultSchedulingPolicy = SchedulingPolicy.DEFAULT_POLICY;
+    placementPolicy = QueuePlacementPolicy.fromConfiguration(conf,
+        new HashSet<String>());
+    queueNames = new HashSet<String>();
+  }
+  
+  /**
+   * Get the ACLs associated with this queue. If a given ACL is not explicitly
+   * configured, include the default value for that ACL.  The default for the
+   * root queue is everybody ("*") and the default for all other queues is
+   * nobody ("")
+   */
+  public AccessControlList getQueueAcl(String queue, QueueACL operation) {
+    Map<QueueACL, AccessControlList> queueAcls = this.queueAcls.get(queue);
+    if (queueAcls != null) {
+      AccessControlList operationAcl = queueAcls.get(operation);
+      if (operationAcl != null) {
+        return operationAcl;
+      }
+    }
+    return (queue.equals("root")) ? EVERYBODY_ACL : NOBODY_ACL;
+  }
+  
+  /**
+   * Get a queue's min share preemption timeout, in milliseconds. This is the
+   * time after which jobs in the queue may kill other queues' tasks if they
+   * are below their min share.
+   */
+  public long getMinSharePreemptionTimeout(String queueName) {
+    Long minSharePreemptionTimeout = minSharePreemptionTimeouts.get(queueName);
+    return (minSharePreemptionTimeout == null) ? defaultMinSharePreemptionTimeout
+        : minSharePreemptionTimeout;
+  }
+  
+  /**
+   * Get the fair share preemption, in milliseconds. This is the time
+   * after which any job may kill other jobs' tasks if it is below half
+   * its fair share.
+   */
+  public long getFairSharePreemptionTimeout() {
+    return fairSharePreemptionTimeout;
+  }
+  
+  public ResourceWeights getQueueWeight(String queue) {
+    ResourceWeights weight = queueWeights.get(queue);
+    return (weight == null) ? ResourceWeights.NEUTRAL : weight;
+  }
+  
+  public int getUserMaxApps(String user) {
+    Integer maxApps = userMaxApps.get(user);
+    return (maxApps == null) ? userMaxAppsDefault : maxApps;
+  }
+
+  public int getQueueMaxApps(String queue) {
+    Integer maxApps = queueMaxApps.get(queue);
+    return (maxApps == null) ? queueMaxAppsDefault : maxApps;
+  }
+  
+  /**
+   * Get the minimum resource allocation for the given queue.
+   * @return the cap set on this queue, or 0 if not set.
+   */
+  public Resource getMinResources(String queue) {
+    Resource minQueueResource = minQueueResources.get(queue);
+    return (minQueueResource == null) ? Resources.none() : minQueueResource;
+  }
+
+  /**
+   * Get the maximum resource allocation for the given queue.
+   * @return the cap set on this queue, or Integer.MAX_VALUE if not set.
+   */
+
+  public Resource getMaxResources(String queueName) {
+    Resource maxQueueResource = maxQueueResources.get(queueName);
+    return (maxQueueResource == null) ? Resources.unbounded() : maxQueueResource;
+  }
+  
+  public boolean hasAccess(String queueName, QueueACL acl,
+      UserGroupInformation user) {
+    int lastPeriodIndex = queueName.length();
+    while (lastPeriodIndex != -1) {
+      String queue = queueName.substring(0, lastPeriodIndex);
+      if (getQueueAcl(queue, acl).isUserAllowed(user)) {
+        return true;
+      }
+
+      lastPeriodIndex = queueName.lastIndexOf('.', lastPeriodIndex - 1);
+    }
+    
+    return false;
+  }
+  
+  public SchedulingPolicy getSchedulingPolicy(String queueName) {
+    SchedulingPolicy policy = schedulingPolicies.get(queueName);
+    return (policy == null) ? defaultSchedulingPolicy : policy;
+  }
+  
+  public SchedulingPolicy getDefaultSchedulingPolicy() {
+    return defaultSchedulingPolicy;
+  }
+  
+  public Set<String> getQueueNames() {
+    return queueNames;
+  }
+  
+  public QueuePlacementPolicy getPlacementPolicy() {
+    return placementPolicy;
+  }
+}

+ 398 - 0
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

@@ -0,0 +1,398 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.QueueACL;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+import org.w3c.dom.Text;
+import org.xml.sax.SAXException;
+
+import com.google.common.annotations.VisibleForTesting;
+
+@Public
+@Unstable
+public class AllocationFileLoaderService extends AbstractService {
+  
+  public static final Log LOG = LogFactory.getLog(
+      AllocationFileLoaderService.class.getName());
+  
+  /** Time to wait between checks of the allocation file */
+  public static final long ALLOC_RELOAD_INTERVAL_MS = 10 * 1000;
+
+  /**
+   * Time to wait after the allocation has been modified before reloading it
+   * (this is done to prevent loading a file that hasn't been fully written).
+   */
+  public static final long ALLOC_RELOAD_WAIT_MS = 5 * 1000;
+  
+  private final Clock clock;
+
+  private long lastSuccessfulReload; // Last time we successfully reloaded queues
+  private boolean lastReloadAttemptFailed = false;
+  
+  // Path to XML file containing allocations. 
+  private File allocFile;
+  
+  private Listener reloadListener;
+  
+  @VisibleForTesting
+  long reloadIntervalMs = ALLOC_RELOAD_INTERVAL_MS;
+  
+  private Thread reloadThread;
+  private volatile boolean running = true;
+  
+  public AllocationFileLoaderService() {
+    this(new SystemClock());
+  }
+  
+  public AllocationFileLoaderService(Clock clock) {
+    super(AllocationFileLoaderService.class.getName());
+    this.clock = clock;
+    
+  }
+  
+  @Override
+  public void init(Configuration conf) {
+    this.allocFile = getAllocationFile(conf);
+    super.init(conf);
+  }
+  
+  @Override
+  public void start() {
+    if (allocFile == null) {
+      return;
+    }
+    reloadThread = new Thread() {
+      public void run() {
+        while (running) {
+          long time = clock.getTime();
+          long lastModified = allocFile.lastModified();
+          if (lastModified > lastSuccessfulReload &&
+              time > lastModified + ALLOC_RELOAD_WAIT_MS) {
+            try {
+              reloadAllocations();
+            } catch (Exception ex) {
+              if (!lastReloadAttemptFailed) {
+                LOG.error("Failed to reload fair scheduler config file - " +
+                    "will use existing allocations.", ex);
+              }
+              lastReloadAttemptFailed = true;
+            }
+          } else if (lastModified == 0l) {
+            if (!lastReloadAttemptFailed) {
+              LOG.warn("Failed to reload fair scheduler config file because" +
+                  " last modified returned 0. File exists: " + allocFile.exists());
+            }
+            lastReloadAttemptFailed = true;
+          }
+          try {
+            Thread.sleep(reloadIntervalMs);
+          } catch (InterruptedException ex) {
+            LOG.info("Interrupted while waiting to reload alloc configuration");
+          }
+        }
+      }
+    };
+    reloadThread.setName("AllocationFileReloader");
+    reloadThread.setDaemon(true);
+    reloadThread.start();
+    super.start();
+  }
+  
+  @Override
+  public void stop() {
+    running = false;
+    reloadThread.interrupt();
+    super.stop();
+  }
+  
+  /**
+   * Path to XML file containing allocations. If the
+   * path is relative, it is searched for in the
+   * classpath, but loaded like a regular File.
+   */
+  public File getAllocationFile(Configuration conf) {
+    String allocFilePath = conf.get(FairSchedulerConfiguration.ALLOCATION_FILE,
+        FairSchedulerConfiguration.DEFAULT_ALLOCATION_FILE);
+    File allocFile = new File(allocFilePath);
+    if (!allocFile.isAbsolute()) {
+      URL url = Thread.currentThread().getContextClassLoader()
+          .getResource(allocFilePath);
+      if (url == null) {
+        LOG.warn(allocFilePath + " not found on the classpath.");
+        allocFile = null;
+      } else if (!url.getProtocol().equalsIgnoreCase("file")) {
+        throw new RuntimeException("Allocation file " + url
+            + " found on the classpath is not on the local filesystem.");
+      } else {
+        allocFile = new File(url.getPath());
+      }
+    }
+    return allocFile;
+  }
+  
+  public synchronized void setReloadListener(Listener reloadListener) {
+    this.reloadListener = reloadListener;
+  }
+  
+  /**
+   * Updates the allocation list from the allocation config file. This file is
+   * expected to be in the XML format specified in the design doc.
+   *
+   * @throws IOException if the config file cannot be read.
+   * @throws AllocationConfigurationException if allocations are invalid.
+   * @throws ParserConfigurationException if XML parser is misconfigured.
+   * @throws SAXException if config file is malformed.
+   */
+  public synchronized void reloadAllocations() throws IOException,
+      ParserConfigurationException, SAXException, AllocationConfigurationException {
+    if (allocFile == null) {
+      return;
+    }
+    LOG.info("Loading allocation file " + allocFile);
+    // Create some temporary hashmaps to hold the new allocs, and we only save
+    // them in our fields if we have parsed the entire allocs file successfully.
+    Map<String, Resource> minQueueResources = new HashMap<String, Resource>();
+    Map<String, Resource> maxQueueResources = new HashMap<String, Resource>();
+    Map<String, Integer> queueMaxApps = new HashMap<String, Integer>();
+    Map<String, Integer> userMaxApps = new HashMap<String, Integer>();
+    Map<String, ResourceWeights> queueWeights = new HashMap<String, ResourceWeights>();
+    Map<String, SchedulingPolicy> queuePolicies = new HashMap<String, SchedulingPolicy>();
+    Map<String, Long> minSharePreemptionTimeouts = new HashMap<String, Long>();
+    Map<String, Map<QueueACL, AccessControlList>> queueAcls =
+        new HashMap<String, Map<QueueACL, AccessControlList>>();
+    int userMaxAppsDefault = Integer.MAX_VALUE;
+    int queueMaxAppsDefault = Integer.MAX_VALUE;
+    long fairSharePreemptionTimeout = Long.MAX_VALUE;
+    long defaultMinSharePreemptionTimeout = Long.MAX_VALUE;
+    SchedulingPolicy defaultSchedPolicy = SchedulingPolicy.DEFAULT_POLICY;
+    
+    QueuePlacementPolicy newPlacementPolicy = null;
+
+    // Remember all queue names so we can display them on web UI, etc.
+    Set<String> queueNamesInAllocFile = new HashSet<String>();
+
+    // Read and parse the allocations file.
+    DocumentBuilderFactory docBuilderFactory =
+      DocumentBuilderFactory.newInstance();
+    docBuilderFactory.setIgnoringComments(true);
+    DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
+    Document doc = builder.parse(allocFile);
+    Element root = doc.getDocumentElement();
+    if (!"allocations".equals(root.getTagName()))
+      throw new AllocationConfigurationException("Bad fair scheduler config " +
+          "file: top-level element not <allocations>");
+    NodeList elements = root.getChildNodes();
+    List<Element> queueElements = new ArrayList<Element>();
+    Element placementPolicyElement = null;
+    for (int i = 0; i < elements.getLength(); i++) {
+      Node node = elements.item(i);
+      if (node instanceof Element) {
+        Element element = (Element)node;
+        if ("queue".equals(element.getTagName()) ||
+          "pool".equals(element.getTagName())) {
+          queueElements.add(element);
+        } else if ("user".equals(element.getTagName())) {
+          String userName = element.getAttribute("name");
+          NodeList fields = element.getChildNodes();
+          for (int j = 0; j < fields.getLength(); j++) {
+            Node fieldNode = fields.item(j);
+            if (!(fieldNode instanceof Element))
+              continue;
+            Element field = (Element) fieldNode;
+            if ("maxRunningApps".equals(field.getTagName())) {
+              String text = ((Text)field.getFirstChild()).getData().trim();
+              int val = Integer.parseInt(text);
+              userMaxApps.put(userName, val);
+            }
+          }
+        } else if ("userMaxAppsDefault".equals(element.getTagName())) {
+          String text = ((Text)element.getFirstChild()).getData().trim();
+          int val = Integer.parseInt(text);
+          userMaxAppsDefault = val;
+        } else if ("fairSharePreemptionTimeout".equals(element.getTagName())) {
+          String text = ((Text)element.getFirstChild()).getData().trim();
+          long val = Long.parseLong(text) * 1000L;
+          fairSharePreemptionTimeout = val;
+        } else if ("defaultMinSharePreemptionTimeout".equals(element.getTagName())) {
+          String text = ((Text)element.getFirstChild()).getData().trim();
+          long val = Long.parseLong(text) * 1000L;
+          defaultMinSharePreemptionTimeout = val;
+        } else if ("queueMaxAppsDefault".equals(element.getTagName())) {
+          String text = ((Text)element.getFirstChild()).getData().trim();
+          int val = Integer.parseInt(text);
+          queueMaxAppsDefault = val;
+        } else if ("defaultQueueSchedulingPolicy".equals(element.getTagName())
+            || "defaultQueueSchedulingMode".equals(element.getTagName())) {
+          String text = ((Text)element.getFirstChild()).getData().trim();
+          defaultSchedPolicy = SchedulingPolicy.parse(text);
+        } else if ("queuePlacementPolicy".equals(element.getTagName())) {
+          placementPolicyElement = element;
+        } else {
+          LOG.warn("Bad element in allocations file: " + element.getTagName());
+        }
+      }
+    }
+    
+    // Load queue elements.  A root queue can either be included or omitted.  If
+    // it's included, all other queues must be inside it.
+    for (Element element : queueElements) {
+      String parent = "root";
+      if (element.getAttribute("name").equalsIgnoreCase("root")) {
+        if (queueElements.size() > 1) {
+          throw new AllocationConfigurationException("If configuring root queue,"
+              + " no other queues can be placed alongside it.");
+        }
+        parent = null;
+      }
+      loadQueue(parent, element, minQueueResources, maxQueueResources, queueMaxApps,
+          userMaxApps, queueWeights, queuePolicies, minSharePreemptionTimeouts,
+          queueAcls, queueNamesInAllocFile);
+    }
+    
+    // Load placement policy and pass it configured queues
+    Configuration conf = getConfig();
+    if (placementPolicyElement != null) {
+      newPlacementPolicy = QueuePlacementPolicy.fromXml(placementPolicyElement,
+          queueNamesInAllocFile, conf);
+    } else {
+      newPlacementPolicy = QueuePlacementPolicy.fromConfiguration(conf,
+          queueNamesInAllocFile);
+    }
+    
+    AllocationConfiguration info = new AllocationConfiguration(minQueueResources, maxQueueResources,
+        queueMaxApps, userMaxApps, queueWeights, userMaxAppsDefault,
+        queueMaxAppsDefault, queuePolicies, defaultSchedPolicy, minSharePreemptionTimeouts,
+        queueAcls, fairSharePreemptionTimeout, defaultMinSharePreemptionTimeout,
+        newPlacementPolicy, queueNamesInAllocFile);
+    
+    lastSuccessfulReload = clock.getTime();
+    lastReloadAttemptFailed = false;
+
+    reloadListener.onReload(info);
+  }
+  
+  /**
+   * Loads a queue from a queue element in the configuration file
+   */
+  private void loadQueue(String parentName, Element element, Map<String, Resource> minQueueResources,
+      Map<String, Resource> maxQueueResources, Map<String, Integer> queueMaxApps,
+      Map<String, Integer> userMaxApps, Map<String, ResourceWeights> queueWeights,
+      Map<String, SchedulingPolicy> queuePolicies,
+      Map<String, Long> minSharePreemptionTimeouts,
+      Map<String, Map<QueueACL, AccessControlList>> queueAcls, Set<String> queueNamesInAllocFile) 
+      throws AllocationConfigurationException {
+    String queueName = element.getAttribute("name");
+    if (parentName != null) {
+      queueName = parentName + "." + queueName;
+    }
+    Map<QueueACL, AccessControlList> acls =
+        new HashMap<QueueACL, AccessControlList>();
+    NodeList fields = element.getChildNodes();
+    boolean isLeaf = true;
+
+    for (int j = 0; j < fields.getLength(); j++) {
+      Node fieldNode = fields.item(j);
+      if (!(fieldNode instanceof Element))
+        continue;
+      Element field = (Element) fieldNode;
+      if ("minResources".equals(field.getTagName())) {
+        String text = ((Text)field.getFirstChild()).getData().trim();
+        Resource val = FairSchedulerConfiguration.parseResourceConfigValue(text);
+        minQueueResources.put(queueName, val);
+      } else if ("maxResources".equals(field.getTagName())) {
+        String text = ((Text)field.getFirstChild()).getData().trim();
+        Resource val = FairSchedulerConfiguration.parseResourceConfigValue(text);
+        maxQueueResources.put(queueName, val);
+      } else if ("maxRunningApps".equals(field.getTagName())) {
+        String text = ((Text)field.getFirstChild()).getData().trim();
+        int val = Integer.parseInt(text);
+        queueMaxApps.put(queueName, val);
+      } else if ("weight".equals(field.getTagName())) {
+        String text = ((Text)field.getFirstChild()).getData().trim();
+        double val = Double.parseDouble(text);
+        queueWeights.put(queueName, new ResourceWeights((float)val));
+      } else if ("minSharePreemptionTimeout".equals(field.getTagName())) {
+        String text = ((Text)field.getFirstChild()).getData().trim();
+        long val = Long.parseLong(text) * 1000L;
+        minSharePreemptionTimeouts.put(queueName, val);
+      } else if ("schedulingPolicy".equals(field.getTagName())
+          || "schedulingMode".equals(field.getTagName())) {
+        String text = ((Text)field.getFirstChild()).getData().trim();
+        SchedulingPolicy policy = SchedulingPolicy.parse(text);
+        queuePolicies.put(queueName, policy);
+      } else if ("aclSubmitApps".equals(field.getTagName())) {
+        String text = ((Text)field.getFirstChild()).getData();
+        acls.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList(text));
+      } else if ("aclAdministerApps".equals(field.getTagName())) {
+        String text = ((Text)field.getFirstChild()).getData();
+        acls.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList(text));
+      } else if ("queue".endsWith(field.getTagName()) || 
+          "pool".equals(field.getTagName())) {
+        loadQueue(queueName, field, minQueueResources, maxQueueResources,
+            queueMaxApps, userMaxApps, queueWeights, queuePolicies,
+            minSharePreemptionTimeouts,
+            queueAcls, queueNamesInAllocFile);
+        isLeaf = false;
+      }
+    }
+    if (isLeaf) {
+      queueNamesInAllocFile.add(queueName);
+    }
+    queueAcls.put(queueName, acls);
+    if (maxQueueResources.containsKey(queueName) && minQueueResources.containsKey(queueName)
+        && !Resources.fitsIn(minQueueResources.get(queueName),
+            maxQueueResources.get(queueName))) {
+      LOG.warn(String.format("Queue %s has max resources %d less than min resources %d",
+          queueName, maxQueueResources.get(queueName), minQueueResources.get(queueName)));
+    }
+  }
+  
+  public interface Listener {
+    public void onReload(AllocationConfiguration info);
+  }
+}

+ 4 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java

@@ -46,19 +46,15 @@ public class FSLeafQueue extends FSQueue {
   private final List<AppSchedulable> nonRunnableAppScheds =
       new ArrayList<AppSchedulable>();
   
-  private final FairScheduler scheduler;
-  private final QueueManager queueMgr;
   private Resource demand = Resources.createResource(0);
   
   // Variables used for preemption
   private long lastTimeAtMinShare;
   private long lastTimeAtHalfFairShare;
   
-  public FSLeafQueue(String name, QueueManager queueMgr, FairScheduler scheduler,
+  public FSLeafQueue(String name, FairScheduler scheduler,
       FSParentQueue parent) {
-    super(name, queueMgr, scheduler, parent);
-    this.scheduler = scheduler;
-    this.queueMgr = queueMgr;
+    super(name, scheduler, parent);
     this.lastTimeAtMinShare = scheduler.getClock().getTime();
     this.lastTimeAtHalfFairShare = scheduler.getClock().getTime();
   }
@@ -145,7 +141,8 @@ public class FSLeafQueue extends FSQueue {
   public void updateDemand() {
     // Compute demand by iterating through apps in the queue
     // Limit demand to maxResources
-    Resource maxRes = queueMgr.getMaxResources(getName());
+    Resource maxRes = scheduler.getAllocationConfiguration()
+        .getMaxResources(getName());
     demand = Resources.createResource(0);
     for (AppSchedulable sched : runnableAppScheds) {
       if (Resources.equals(demand, maxRes)) {

+ 6 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java

@@ -41,14 +41,12 @@ public class FSParentQueue extends FSQueue {
 
   private final List<FSQueue> childQueues = 
       new ArrayList<FSQueue>();
-  private final QueueManager queueMgr;
   private Resource demand = Resources.createResource(0);
   private int runnableApps;
   
-  public FSParentQueue(String name, QueueManager queueMgr, FairScheduler scheduler,
+  public FSParentQueue(String name, FairScheduler scheduler,
       FSParentQueue parent) {
-    super(name, queueMgr, scheduler, parent);
-    this.queueMgr = queueMgr;
+    super(name, scheduler, parent);
   }
   
   public void addChildQueue(FSQueue child) {
@@ -82,7 +80,8 @@ public class FSParentQueue extends FSQueue {
   public void updateDemand() {
     // Compute demand by iterating through apps in the queue
     // Limit demand to maxResources
-    Resource maxRes = queueMgr.getMaxResources(getName());
+    Resource maxRes = scheduler.getAllocationConfiguration()
+        .getMaxResources(getName());
     demand = Resources.createResource(0);
     for (FSQueue childQueue : childQueues) {
       childQueue.updateDemand();
@@ -164,8 +163,8 @@ public class FSParentQueue extends FSQueue {
   public void setPolicy(SchedulingPolicy policy)
       throws AllocationConfigurationException {
     boolean allowed =
-        SchedulingPolicy.isApplicableTo(policy, (this == queueMgr
-            .getRootQueue()) ? SchedulingPolicy.DEPTH_ROOT
+        SchedulingPolicy.isApplicableTo(policy, (parent == null)
+            ? SchedulingPolicy.DEPTH_ROOT
             : SchedulingPolicy.DEPTH_INTERMEDIATE);
     if (!allowed) {
       throwPolicyDoesnotApplyException(policy);

+ 8 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java

@@ -39,20 +39,17 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 @Unstable
 public abstract class FSQueue extends Schedulable implements Queue {
   private final String name;
-  private final QueueManager queueMgr;
-  private final FairScheduler scheduler;
+  protected final FairScheduler scheduler;
   private final FSQueueMetrics metrics;
   
   protected final FSParentQueue parent;
   protected final RecordFactory recordFactory =
       RecordFactoryProvider.getRecordFactory(null);
   
-  protected SchedulingPolicy policy = SchedulingPolicy.getDefault();
+  protected SchedulingPolicy policy = SchedulingPolicy.DEFAULT_POLICY;
 
-  public FSQueue(String name, QueueManager queueMgr, 
-      FairScheduler scheduler, FSParentQueue parent) {
+  public FSQueue(String name, FairScheduler scheduler, FSParentQueue parent) {
     this.name = name;
-    this.queueMgr = queueMgr;
     this.scheduler = scheduler;
     this.metrics = FSQueueMetrics.forQueue(getName(), parent, true, scheduler.getConf());
     metrics.setMinShare(getMinShare());
@@ -88,17 +85,17 @@ public abstract class FSQueue extends Schedulable implements Queue {
 
   @Override
   public ResourceWeights getWeights() {
-    return queueMgr.getQueueWeight(getName());
+    return scheduler.getAllocationConfiguration().getQueueWeight(getName());
   }
   
   @Override
   public Resource getMinShare() {
-    return queueMgr.getMinResources(getName());
+    return scheduler.getAllocationConfiguration().getMinResources(getName());
   }
   
   @Override
   public Resource getMaxShare() {
-    return queueMgr.getMaxResources(getName());
+    return scheduler.getAllocationConfiguration().getMaxResources(getName());
   }
 
   @Override
@@ -148,13 +145,7 @@ public abstract class FSQueue extends Schedulable implements Queue {
   }
   
   public boolean hasAccess(QueueACL acl, UserGroupInformation user) {
-    // Check if the leaf-queue allows access
-    if (queueMgr.getQueueAcl(getName(), acl).isUserAllowed(user)) {
-      return true;
-    }
-
-    // Check if parent-queue allows access
-    return parent != null && parent.hasAccess(acl, user);
+    return scheduler.getAllocationConfiguration().hasAccess(name, acl, user);
   }
   
   /**
@@ -181,7 +172,7 @@ public abstract class FSQueue extends Schedulable implements Queue {
    */
   protected boolean assignContainerPreCheck(FSSchedulerNode node) {
     if (!Resources.fitsIn(getResourceUsage(),
-        queueMgr.getMaxResources(getName()))
+        scheduler.getAllocationConfiguration().getMaxResources(getName()))
         || node.getReservedContainer() != null) {
       return false;
     }

+ 65 - 29
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java

@@ -192,11 +192,16 @@ public class FairScheduler implements ResourceScheduler {
 
   @VisibleForTesting
   final MaxRunningAppsEnforcer maxRunningEnforcer;
+
+  private AllocationFileLoaderService allocsLoader;
+  @VisibleForTesting
+  AllocationConfiguration allocConf;
   
   public FairScheduler() {
     clock = new SystemClock();
+    allocsLoader = new AllocationFileLoaderService();
     queueMgr = new QueueManager(this);
-    maxRunningEnforcer = new MaxRunningAppsEnforcer(queueMgr);
+    maxRunningEnforcer = new MaxRunningAppsEnforcer(this);
   }
 
   private void validateConf(Configuration conf) {
@@ -275,7 +280,6 @@ public class FairScheduler implements ResourceScheduler {
    * required resources per job.
    */
   protected synchronized void update() {
-    queueMgr.reloadAllocsIfNecessary(); // Relaod alloc file
     updatePreemptionVariables(); // Determine if any queues merit preemption
 
     FSQueue rootQueue = queueMgr.getRootQueue();
@@ -480,8 +484,8 @@ public class FairScheduler implements ResourceScheduler {
    */
   protected Resource resToPreempt(FSLeafQueue sched, long curTime) {
     String queue = sched.getName();
-    long minShareTimeout = queueMgr.getMinSharePreemptionTimeout(queue);
-    long fairShareTimeout = queueMgr.getFairSharePreemptionTimeout();
+    long minShareTimeout = allocConf.getMinSharePreemptionTimeout(queue);
+    long fairShareTimeout = allocConf.getFairSharePreemptionTimeout();
     Resource resDueToMinShare = Resources.none();
     Resource resDueToFairShare = Resources.none();
     if (curTime - sched.getLastTimeAtMinShare() > minShareTimeout) {
@@ -650,8 +654,8 @@ public class FairScheduler implements ResourceScheduler {
   FSLeafQueue assignToQueue(RMApp rmApp, String queueName, String user) {
     FSLeafQueue queue = null;
     try {
-      QueuePlacementPolicy policy = queueMgr.getPlacementPolicy();
-      queueName = policy.assignAppToQueue(queueName, user);
+      QueuePlacementPolicy placementPolicy = allocConf.getPlacementPolicy();
+      queueName = placementPolicy.assignAppToQueue(queueName, user);
       if (queueName == null) {
         return null;
       }
@@ -1128,27 +1132,27 @@ public class FairScheduler implements ResourceScheduler {
   @Override
   public synchronized void reinitialize(Configuration conf, RMContext rmContext)
       throws IOException {
-    this.conf = new FairSchedulerConfiguration(conf);
-    validateConf(this.conf);
-    minimumAllocation = this.conf.getMinimumAllocation();
-    maximumAllocation = this.conf.getMaximumAllocation();
-    incrAllocation = this.conf.getIncrementAllocation();
-    continuousSchedulingEnabled = this.conf.isContinuousSchedulingEnabled();
-    continuousSchedulingSleepMs =
-            this.conf.getContinuousSchedulingSleepMs();
-    nodeLocalityThreshold = this.conf.getLocalityThresholdNode();
-    rackLocalityThreshold = this.conf.getLocalityThresholdRack();
-    nodeLocalityDelayMs = this.conf.getLocalityDelayNodeMs();
-    rackLocalityDelayMs = this.conf.getLocalityDelayRackMs();
-    preemptionEnabled = this.conf.getPreemptionEnabled();
-    assignMultiple = this.conf.getAssignMultiple();
-    maxAssign = this.conf.getMaxAssign();
-    sizeBasedWeight = this.conf.getSizeBasedWeight();
-    preemptionInterval = this.conf.getPreemptionInterval();
-    waitTimeBeforeKill = this.conf.getWaitTimeBeforeKill();
-    usePortForNodeName = this.conf.getUsePortForNodeName();
-    
     if (!initialized) {
+      this.conf = new FairSchedulerConfiguration(conf);
+      validateConf(this.conf);
+      minimumAllocation = this.conf.getMinimumAllocation();
+      maximumAllocation = this.conf.getMaximumAllocation();
+      incrAllocation = this.conf.getIncrementAllocation();
+      continuousSchedulingEnabled = this.conf.isContinuousSchedulingEnabled();
+      continuousSchedulingSleepMs =
+              this.conf.getContinuousSchedulingSleepMs();
+      nodeLocalityThreshold = this.conf.getLocalityThresholdNode();
+      rackLocalityThreshold = this.conf.getLocalityThresholdRack();
+      nodeLocalityDelayMs = this.conf.getLocalityDelayNodeMs();
+      rackLocalityDelayMs = this.conf.getLocalityDelayRackMs();
+      preemptionEnabled = this.conf.getPreemptionEnabled();
+      assignMultiple = this.conf.getAssignMultiple();
+      maxAssign = this.conf.getMaxAssign();
+      sizeBasedWeight = this.conf.getSizeBasedWeight();
+      preemptionInterval = this.conf.getPreemptionInterval();
+      waitTimeBeforeKill = this.conf.getWaitTimeBeforeKill();
+      usePortForNodeName = this.conf.getUsePortForNodeName();
+      
       rootMetrics = FSQueueMetrics.forQueue("root", null, true, conf);
       this.rmContext = rmContext;
       this.eventLog = new FairSchedulerEventLog();
@@ -1156,8 +1160,9 @@ public class FairScheduler implements ResourceScheduler {
 
       initialized = true;
 
+      allocConf = new AllocationConfiguration(conf);
       try {
-        queueMgr.initialize();
+        queueMgr.initialize(conf);
       } catch (Exception e) {
         throw new IOException("Failed to start FairScheduler", e);
       }
@@ -1181,12 +1186,24 @@ public class FairScheduler implements ResourceScheduler {
         schedulingThread.setDaemon(true);
         schedulingThread.start();
       }
-    } else {
+      
+      allocsLoader.init(conf);
+      allocsLoader.setReloadListener(new AllocationReloadListener());
+      // If we fail to load allocations file on initialize, we want to fail
+      // immediately.  After a successful load, exceptions on future reloads
+      // will just result in leaving things as they are.
       try {
-        queueMgr.reloadAllocs();
+        allocsLoader.reloadAllocations();
       } catch (Exception e) {
         throw new IOException("Failed to initialize FairScheduler", e);
       }
+      allocsLoader.start();
+    } else {
+      try {
+        allocsLoader.reloadAllocations();
+      } catch (Exception e) {
+        LOG.error("Failed to reload allocations file", e);
+      }
     }
   }
 
@@ -1230,5 +1247,24 @@ public class FairScheduler implements ResourceScheduler {
     }
     return queue.hasAccess(acl, callerUGI);
   }
+  
+  public AllocationConfiguration getAllocationConfiguration() {
+    return allocConf;
+  }
+  
+  private class AllocationReloadListener implements
+      AllocationFileLoaderService.Listener {
+
+    @Override
+    public void onReload(AllocationConfiguration queueInfo) {
+      // Commit the reload; also create any queue defined in the alloc file
+      // if it does not already exist, so it can be displayed on the web UI.
+      synchronized (FairScheduler.this) {
+        allocConf = queueInfo;
+        allocConf.getDefaultSchedulingPolicy().initialize(clusterCapacity);
+        queueMgr.updateAllocationConfiguration(allocConf);
+      }
+    }
+  }
 
 }

+ 2 - 33
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

@@ -18,7 +18,8 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
 import java.io.File;
-import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -151,14 +152,6 @@ public class FairSchedulerConfiguration extends Configuration {
     return Resources.createResource(incrementMemory, incrementCores);
   }
   
-  public boolean getAllowUndeclaredPools() {
-    return getBoolean(ALLOW_UNDECLARED_POOLS, DEFAULT_ALLOW_UNDECLARED_POOLS);
-  }
-
-  public boolean getUserAsDefaultQueue() {
-    return getBoolean(USER_AS_DEFAULT_QUEUE, DEFAULT_USER_AS_DEFAULT_QUEUE);
-  }
-
   public float getLocalityThresholdNode() {
     return getFloat(LOCALITY_THRESHOLD_NODE, DEFAULT_LOCALITY_THRESHOLD_NODE);
   }
@@ -199,30 +192,6 @@ public class FairSchedulerConfiguration extends Configuration {
     return getBoolean(SIZE_BASED_WEIGHT, DEFAULT_SIZE_BASED_WEIGHT);
   }
 
-  /**
-   * Path to XML file containing allocations. If the
-   * path is relative, it is searched for in the
-   * classpath, but loaded like a regular File.
-   */
-  public File getAllocationFile() {
-    String allocFilePath = get(ALLOCATION_FILE, DEFAULT_ALLOCATION_FILE);
-    File allocFile = new File(allocFilePath);
-    if (!allocFile.isAbsolute()) {
-      URL url = Thread.currentThread().getContextClassLoader()
-          .getResource(allocFilePath);
-      if (url == null) {
-        LOG.warn(allocFilePath + " not found on the classpath.");
-        allocFile = null;
-      } else if (!url.getProtocol().equalsIgnoreCase("file")) {
-        throw new RuntimeException("Allocation file " + url
-            + " found on the classpath is not on the local filesystem.");
-      } else {
-        allocFile = new File(url.getPath());
-      }
-    }
-    return allocFile;
-  }
-
   public String getEventlogDir() {
     return get(EVENT_LOG_DIR, new File(System.getProperty("hadoop.log.dir",
     		"/tmp/")).getAbsolutePath() + File.separator + "fairscheduler");

+ 13 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/MaxRunningAppsEnforcer.java

@@ -33,15 +33,15 @@ import com.google.common.collect.ListMultimap;
  * constraints
  */
 public class MaxRunningAppsEnforcer {
-  private final QueueManager queueMgr;
+  private final FairScheduler scheduler;
 
   // Tracks the number of running applications by user.
   private final Map<String, Integer> usersNumRunnableApps;
   @VisibleForTesting
   final ListMultimap<String, AppSchedulable> usersNonRunnableApps;
 
-  public MaxRunningAppsEnforcer(QueueManager queueMgr) {
-    this.queueMgr = queueMgr;
+  public MaxRunningAppsEnforcer(FairScheduler scheduler) {
+    this.scheduler = scheduler;
     this.usersNumRunnableApps = new HashMap<String, Integer>();
     this.usersNonRunnableApps = ArrayListMultimap.create();
   }
@@ -51,16 +51,17 @@ public class MaxRunningAppsEnforcer {
    * maxRunningApps limits.
    */
   public boolean canAppBeRunnable(FSQueue queue, String user) {
+    AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
     Integer userNumRunnable = usersNumRunnableApps.get(user);
     if (userNumRunnable == null) {
       userNumRunnable = 0;
     }
-    if (userNumRunnable >= queueMgr.getUserMaxApps(user)) {
+    if (userNumRunnable >= allocConf.getUserMaxApps(user)) {
       return false;
     }
     // Check queue and all parent queues
     while (queue != null) {
-      int queueMaxApps = queueMgr.getQueueMaxApps(queue.getName());
+      int queueMaxApps = allocConf.getQueueMaxApps(queue.getName());
       if (queue.getNumRunnableApps() >= queueMaxApps) {
         return false;
       }
@@ -107,6 +108,8 @@ public class MaxRunningAppsEnforcer {
    * highest queue that went from having no slack to having slack.
    */
   public void updateRunnabilityOnAppRemoval(FSSchedulerApp app) {
+    AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
+    
     // Update usersRunnableApps
     String user = app.getUser();
     int newUserNumRunning = usersNumRunnableApps.get(user) - 1;
@@ -127,10 +130,10 @@ public class MaxRunningAppsEnforcer {
     // that was at its maxRunningApps before the removal.
     FSLeafQueue queue = app.getQueue();
     FSQueue highestQueueWithAppsNowRunnable = (queue.getNumRunnableApps() ==
-        queueMgr.getQueueMaxApps(queue.getName()) - 1) ? queue : null;
+        allocConf.getQueueMaxApps(queue.getName()) - 1) ? queue : null;
     FSParentQueue parent = queue.getParent();
     while (parent != null) {
-      if (parent.getNumRunnableApps() == queueMgr.getQueueMaxApps(parent
+      if (parent.getNumRunnableApps() == allocConf.getQueueMaxApps(parent
           .getName())) {
         highestQueueWithAppsNowRunnable = parent;
       }
@@ -149,7 +152,7 @@ public class MaxRunningAppsEnforcer {
       gatherPossiblyRunnableAppLists(highestQueueWithAppsNowRunnable,
           appsNowMaybeRunnable);
     }
-    if (newUserNumRunning == queueMgr.getUserMaxApps(user) - 1) {
+    if (newUserNumRunning == allocConf.getUserMaxApps(user) - 1) {
       List<AppSchedulable> userWaitingApps = usersNonRunnableApps.get(user);
       if (userWaitingApps != null) {
         appsNowMaybeRunnable.add(userWaitingApps);
@@ -200,7 +203,8 @@ public class MaxRunningAppsEnforcer {
    */
   private void gatherPossiblyRunnableAppLists(FSQueue queue,
       List<List<AppSchedulable>> appLists) {
-    if (queue.getNumRunnableApps() < queueMgr.getQueueMaxApps(queue.getName())) {
+    if (queue.getNumRunnableApps() < scheduler.getAllocationConfiguration()
+        .getQueueMaxApps(queue.getName())) {
       if (queue instanceof FSLeafQueue) {
         appLists.add(((FSLeafQueue)queue).getNonRunnableAppSchedulables());
       } else {

+ 36 - 504
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java

@@ -18,20 +18,14 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
-import java.io.File;
 import java.io.IOException;
-import java.net.URL;
-import java.net.URLConnection;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.CopyOnWriteArrayList;
 
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
 
 import org.apache.commons.logging.Log;
@@ -39,21 +33,9 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.authorize.AccessControlList;
-import org.apache.hadoop.yarn.api.records.QueueACL;
-import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
-import org.apache.hadoop.yarn.util.resource.Resources;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
-import org.w3c.dom.Text;
 import org.xml.sax.SAXException;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
  * Maintains a list of queues as well as scheduling parameters for each queue,
  * such as guaranteed share allocations, from the fair scheduler config file.
@@ -67,37 +49,13 @@ public class QueueManager {
 
   public static final String ROOT_QUEUE = "root";
   
-  /** Time to wait between checks of the allocation file */
-  public static final long ALLOC_RELOAD_INTERVAL = 10 * 1000;
-
-  /**
-   * Time to wait after the allocation has been modified before reloading it
-   * (this is done to prevent loading a file that hasn't been fully written).
-   */
-  public static final long ALLOC_RELOAD_WAIT = 5 * 1000;
-  
-  private static final AccessControlList EVERYBODY_ACL = new AccessControlList("*");
-  private static final AccessControlList NOBODY_ACL = new AccessControlList(" ");
-
   private final FairScheduler scheduler;
 
-  // Path to XML file containing allocations. 
-  private File allocFile; 
-
   private final Collection<FSLeafQueue> leafQueues = 
       new CopyOnWriteArrayList<FSLeafQueue>();
   private final Map<String, FSQueue> queues = new HashMap<String, FSQueue>();
   private FSParentQueue rootQueue;
 
-  @VisibleForTesting
-  volatile QueueManagerInfo info = new QueueManagerInfo();
-  @VisibleForTesting
-  volatile QueuePlacementPolicy placementPolicy;
-  
-  private long lastReloadAttempt; // Last time we tried to reload the queues file
-  private long lastSuccessfulReload; // Last time we successfully reloaded queues
-  private boolean lastReloadAttemptFailed = false;
-  
   public QueueManager(FairScheduler scheduler) {
     this.scheduler = scheduler;
   }
@@ -106,45 +64,15 @@ public class QueueManager {
     return rootQueue;
   }
 
-  public void initialize() throws IOException, SAXException,
-      AllocationConfigurationException, ParserConfigurationException {
-    FairSchedulerConfiguration conf = scheduler.getConf();
-    rootQueue = new FSParentQueue("root", this, scheduler, null);
+  public void initialize(Configuration conf) throws IOException,
+      SAXException, AllocationConfigurationException, ParserConfigurationException {
+    rootQueue = new FSParentQueue("root", scheduler, null);
     queues.put(rootQueue.getName(), rootQueue);
     
-    this.allocFile = conf.getAllocationFile();
-    placementPolicy = new QueuePlacementPolicy(getSimplePlacementRules(),
-        new HashSet<String>(), conf);
-    
-    reloadAllocs();
-    lastSuccessfulReload = scheduler.getClock().getTime();
-    lastReloadAttempt = scheduler.getClock().getTime();
     // Create the default queue
     getLeafQueue(YarnConfiguration.DEFAULT_QUEUE_NAME, true);
   }
   
-  public void updatePlacementPolicy(FairSchedulerConfiguration conf) {
-    
-  }
-  
-  /**
-   * Construct simple queue placement policy from allow-undeclared-pools and
-   * user-as-default-queue.
-   */
-  private List<QueuePlacementRule> getSimplePlacementRules() {
-    boolean create = scheduler.getConf().getAllowUndeclaredPools();
-    boolean userAsDefaultQueue = scheduler.getConf().getUserAsDefaultQueue();
-    List<QueuePlacementRule> rules = new ArrayList<QueuePlacementRule>();
-    rules.add(new QueuePlacementRule.Specified().initialize(create, null));
-    if (userAsDefaultQueue) {
-      rules.add(new QueuePlacementRule.User().initialize(create, null));
-    }
-    if (!userAsDefaultQueue || !create) {
-      rules.add(new QueuePlacementRule.Default().initialize(true, null));
-    }
-    return rules;
-  }
-  
   /**
    * Get a queue by name, creating it if the create param is true and is necessary.
    * If the queue is not or can not be a leaf queue, i.e. it already exists as a
@@ -213,17 +141,30 @@ public class QueueManager {
     // queue to create.
     // Now that we know everything worked out, make all the queues
     // and add them to the map.
+    AllocationConfiguration queueConf = scheduler.getAllocationConfiguration();
     FSLeafQueue leafQueue = null;
     for (int i = newQueueNames.size()-1; i >= 0; i--) {
       String queueName = newQueueNames.get(i);
       if (i == 0) {
         // First name added was the leaf queue
-        leafQueue = new FSLeafQueue(name, this, scheduler, parent);
+        leafQueue = new FSLeafQueue(name, scheduler, parent);
+        try {
+          leafQueue.setPolicy(queueConf.getDefaultSchedulingPolicy());
+        } catch (AllocationConfigurationException ex) {
+          LOG.warn("Failed to set default scheduling policy "
+              + queueConf.getDefaultSchedulingPolicy() + " on new leaf queue.", ex);
+        }
         parent.addChildQueue(leafQueue);
         queues.put(leafQueue.getName(), leafQueue);
         leafQueues.add(leafQueue);
       } else {
-        FSParentQueue newParent = new FSParentQueue(queueName, this, scheduler, parent);
+        FSParentQueue newParent = new FSParentQueue(queueName, scheduler, parent);
+        try {
+          newParent.setPolicy(queueConf.getDefaultSchedulingPolicy());
+        } catch (AllocationConfigurationException ex) {
+          LOG.warn("Failed to set default scheduling policy "
+              + queueConf.getDefaultSchedulingPolicy() + " on new parent queue.", ex);
+        }
         parent.addChildQueue(newParent);
         queues.put(newParent.getName(), newParent);
         parent = newParent;
@@ -257,301 +198,6 @@ public class QueueManager {
     }
   }
   
-  public QueuePlacementPolicy getPlacementPolicy() {
-    return placementPolicy;
-  }
-
-  /**
-   * Reload allocations file if it hasn't been loaded in a while
-   */
-  public void reloadAllocsIfNecessary() {
-    long time = scheduler.getClock().getTime();
-    if (time > lastReloadAttempt + ALLOC_RELOAD_INTERVAL) {
-      lastReloadAttempt = time;
-      if (null == allocFile) {
-        return;
-      }
-      try {
-        // Get last modified time of alloc file depending whether it's a String
-        // (for a path name) or an URL (for a classloader resource)
-        long lastModified = allocFile.lastModified();
-        if (lastModified > lastSuccessfulReload &&
-            time > lastModified + ALLOC_RELOAD_WAIT) {
-          reloadAllocs();
-          lastSuccessfulReload = time;
-          lastReloadAttemptFailed = false;
-        }
-      } catch (Exception e) {
-        // Throwing the error further out here won't help - the RPC thread
-        // will catch it and report it in a loop. Instead, just log it and
-        // hope somebody will notice from the log.
-        // We log the error only on the first failure so we don't fill up the
-        // JobTracker's log with these messages.
-        if (!lastReloadAttemptFailed) {
-          LOG.error("Failed to reload fair scheduler config file - " +
-              "will use existing allocations.", e);
-        }
-        lastReloadAttemptFailed = true;
-      }
-    }
-  }
-
-  /**
-   * Updates the allocation list from the allocation config file. This file is
-   * expected to be in the XML format specified in the design doc.
-   *
-   * @throws IOException if the config file cannot be read.
-   * @throws AllocationConfigurationException if allocations are invalid.
-   * @throws ParserConfigurationException if XML parser is misconfigured.
-   * @throws SAXException if config file is malformed.
-   */
-  public void reloadAllocs() throws IOException, ParserConfigurationException,
-      SAXException, AllocationConfigurationException {
-    if (allocFile == null) return;
-    // Create some temporary hashmaps to hold the new allocs, and we only save
-    // them in our fields if we have parsed the entire allocs file successfully.
-    Map<String, Resource> minQueueResources = new HashMap<String, Resource>();
-    Map<String, Resource> maxQueueResources = new HashMap<String, Resource>();
-    Map<String, Integer> queueMaxApps = new HashMap<String, Integer>();
-    Map<String, Integer> userMaxApps = new HashMap<String, Integer>();
-    Map<String, ResourceWeights> queueWeights = new HashMap<String, ResourceWeights>();
-    Map<String, SchedulingPolicy> queuePolicies = new HashMap<String, SchedulingPolicy>();
-    Map<String, Long> minSharePreemptionTimeouts = new HashMap<String, Long>();
-    Map<String, Map<QueueACL, AccessControlList>> queueAcls =
-        new HashMap<String, Map<QueueACL, AccessControlList>>();
-    int userMaxAppsDefault = Integer.MAX_VALUE;
-    int queueMaxAppsDefault = Integer.MAX_VALUE;
-    long fairSharePreemptionTimeout = Long.MAX_VALUE;
-    long defaultMinSharePreemptionTimeout = Long.MAX_VALUE;
-    SchedulingPolicy defaultSchedPolicy = SchedulingPolicy.getDefault();
-    
-    QueuePlacementPolicy newPlacementPolicy = null;
-
-    // Remember all queue names so we can display them on web UI, etc.
-    List<String> queueNamesInAllocFile = new ArrayList<String>();
-
-    // Read and parse the allocations file.
-    DocumentBuilderFactory docBuilderFactory =
-      DocumentBuilderFactory.newInstance();
-    docBuilderFactory.setIgnoringComments(true);
-    DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
-    Document doc = builder.parse(allocFile);
-    Element root = doc.getDocumentElement();
-    if (!"allocations".equals(root.getTagName()))
-      throw new AllocationConfigurationException("Bad fair scheduler config " +
-          "file: top-level element not <allocations>");
-    NodeList elements = root.getChildNodes();
-    List<Element> queueElements = new ArrayList<Element>();
-    Element placementPolicyElement = null;
-    for (int i = 0; i < elements.getLength(); i++) {
-      Node node = elements.item(i);
-      if (node instanceof Element) {
-        Element element = (Element)node;
-        if ("queue".equals(element.getTagName()) ||
-      	  "pool".equals(element.getTagName())) {
-          queueElements.add(element);
-        } else if ("user".equals(element.getTagName())) {
-          String userName = element.getAttribute("name");
-          NodeList fields = element.getChildNodes();
-          for (int j = 0; j < fields.getLength(); j++) {
-            Node fieldNode = fields.item(j);
-            if (!(fieldNode instanceof Element))
-              continue;
-            Element field = (Element) fieldNode;
-            if ("maxRunningApps".equals(field.getTagName())) {
-              String text = ((Text)field.getFirstChild()).getData().trim();
-              int val = Integer.parseInt(text);
-              userMaxApps.put(userName, val);
-            }
-          }
-        } else if ("userMaxAppsDefault".equals(element.getTagName())) {
-          String text = ((Text)element.getFirstChild()).getData().trim();
-          int val = Integer.parseInt(text);
-          userMaxAppsDefault = val;
-        } else if ("fairSharePreemptionTimeout".equals(element.getTagName())) {
-          String text = ((Text)element.getFirstChild()).getData().trim();
-          long val = Long.parseLong(text) * 1000L;
-          fairSharePreemptionTimeout = val;
-        } else if ("defaultMinSharePreemptionTimeout".equals(element.getTagName())) {
-          String text = ((Text)element.getFirstChild()).getData().trim();
-          long val = Long.parseLong(text) * 1000L;
-          defaultMinSharePreemptionTimeout = val;
-        } else if ("queueMaxAppsDefault".equals(element.getTagName())) {
-          String text = ((Text)element.getFirstChild()).getData().trim();
-          int val = Integer.parseInt(text);
-          queueMaxAppsDefault = val;
-        } else if ("defaultQueueSchedulingPolicy".equals(element.getTagName())
-            || "defaultQueueSchedulingMode".equals(element.getTagName())) {
-          String text = ((Text)element.getFirstChild()).getData().trim();
-          SchedulingPolicy.setDefault(text);
-          defaultSchedPolicy = SchedulingPolicy.getDefault();
-        } else if ("queuePlacementPolicy".equals(element.getTagName())) {
-          placementPolicyElement = element;
-        } else {
-          LOG.warn("Bad element in allocations file: " + element.getTagName());
-        }
-      }
-    }
-    
-    // Load queue elements.  A root queue can either be included or omitted.  If
-    // it's included, all other queues must be inside it.
-    for (Element element : queueElements) {
-      String parent = "root";
-      if (element.getAttribute("name").equalsIgnoreCase("root")) {
-        if (queueElements.size() > 1) {
-          throw new AllocationConfigurationException("If configuring root queue,"
-          		+ " no other queues can be placed alongside it.");
-        }
-        parent = null;
-      }
-      loadQueue(parent, element, minQueueResources, maxQueueResources, queueMaxApps,
-          userMaxApps, queueWeights, queuePolicies, minSharePreemptionTimeouts,
-          queueAcls, queueNamesInAllocFile);
-    }
-    
-    // Load placement policy and pass it configured queues
-    if (placementPolicyElement != null) {
-      newPlacementPolicy = QueuePlacementPolicy.fromXml(placementPolicyElement,
-          new HashSet<String>(queueNamesInAllocFile), scheduler.getConf());
-    } else {
-      newPlacementPolicy = new QueuePlacementPolicy(getSimplePlacementRules(),
-          new HashSet<String>(queueNamesInAllocFile), scheduler.getConf());
-    }
-
-    // Commit the reload; also create any queue defined in the alloc file
-    // if it does not already exist, so it can be displayed on the web UI.
-    synchronized (this) {
-      info = new QueueManagerInfo(minQueueResources, maxQueueResources,
-          queueMaxApps, userMaxApps, queueWeights, userMaxAppsDefault,
-          queueMaxAppsDefault, defaultSchedPolicy, minSharePreemptionTimeouts,
-          queueAcls, fairSharePreemptionTimeout, defaultMinSharePreemptionTimeout);
-      placementPolicy = newPlacementPolicy;
-      
-      // Make sure all queues exist
-      for (String name: queueNamesInAllocFile) {
-        getLeafQueue(name, true);
-      }
-      
-      for (FSQueue queue : queues.values()) {
-        // Update queue metrics
-        FSQueueMetrics queueMetrics = queue.getMetrics();
-        queueMetrics.setMinShare(queue.getMinShare());
-        queueMetrics.setMaxShare(queue.getMaxShare());
-        // Set scheduling policies
-        if (queuePolicies.containsKey(queue.getName())) {
-          queue.setPolicy(queuePolicies.get(queue.getName()));
-        } else {
-          queue.setPolicy(SchedulingPolicy.getDefault());
-        }
-      }
- 
-    }
-  }
-  
-  /**
-   * Loads a queue from a queue element in the configuration file
-   */
-  private void loadQueue(String parentName, Element element, Map<String, Resource> minQueueResources,
-      Map<String, Resource> maxQueueResources, Map<String, Integer> queueMaxApps,
-      Map<String, Integer> userMaxApps, Map<String, ResourceWeights> queueWeights,
-      Map<String, SchedulingPolicy> queuePolicies,
-      Map<String, Long> minSharePreemptionTimeouts,
-      Map<String, Map<QueueACL, AccessControlList>> queueAcls, List<String> queueNamesInAllocFile) 
-      throws AllocationConfigurationException {
-    String queueName = element.getAttribute("name");
-    if (parentName != null) {
-      queueName = parentName + "." + queueName;
-    }
-    Map<QueueACL, AccessControlList> acls =
-        new HashMap<QueueACL, AccessControlList>();
-    NodeList fields = element.getChildNodes();
-    boolean isLeaf = true;
-
-    for (int j = 0; j < fields.getLength(); j++) {
-      Node fieldNode = fields.item(j);
-      if (!(fieldNode instanceof Element))
-        continue;
-      Element field = (Element) fieldNode;
-      if ("minResources".equals(field.getTagName())) {
-        String text = ((Text)field.getFirstChild()).getData().trim();
-        Resource val = FairSchedulerConfiguration.parseResourceConfigValue(text);
-        minQueueResources.put(queueName, val);
-      } else if ("maxResources".equals(field.getTagName())) {
-        String text = ((Text)field.getFirstChild()).getData().trim();
-        Resource val = FairSchedulerConfiguration.parseResourceConfigValue(text);
-        maxQueueResources.put(queueName, val);
-      } else if ("maxRunningApps".equals(field.getTagName())) {
-        String text = ((Text)field.getFirstChild()).getData().trim();
-        int val = Integer.parseInt(text);
-        queueMaxApps.put(queueName, val);
-      } else if ("weight".equals(field.getTagName())) {
-        String text = ((Text)field.getFirstChild()).getData().trim();
-        double val = Double.parseDouble(text);
-        queueWeights.put(queueName, new ResourceWeights((float)val));
-      } else if ("minSharePreemptionTimeout".equals(field.getTagName())) {
-        String text = ((Text)field.getFirstChild()).getData().trim();
-        long val = Long.parseLong(text) * 1000L;
-        minSharePreemptionTimeouts.put(queueName, val);
-      } else if ("schedulingPolicy".equals(field.getTagName())
-          || "schedulingMode".equals(field.getTagName())) {
-        String text = ((Text)field.getFirstChild()).getData().trim();
-        SchedulingPolicy policy = SchedulingPolicy.parse(text);
-        policy.initialize(scheduler.getClusterCapacity());
-        queuePolicies.put(queueName, policy);
-      } else if ("aclSubmitApps".equals(field.getTagName())) {
-        String text = ((Text)field.getFirstChild()).getData();
-        acls.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList(text));
-      } else if ("aclAdministerApps".equals(field.getTagName())) {
-        String text = ((Text)field.getFirstChild()).getData();
-        acls.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList(text));
-      } else if ("queue".endsWith(field.getTagName()) || 
-          "pool".equals(field.getTagName())) {
-        loadQueue(queueName, field, minQueueResources, maxQueueResources,
-            queueMaxApps, userMaxApps, queueWeights, queuePolicies,
-            minSharePreemptionTimeouts,
-            queueAcls, queueNamesInAllocFile);
-        isLeaf = false;
-      }
-    }
-    if (isLeaf) {
-      queueNamesInAllocFile.add(queueName);
-    }
-    queueAcls.put(queueName, acls);
-    if (maxQueueResources.containsKey(queueName) && minQueueResources.containsKey(queueName)
-        && !Resources.fitsIn(minQueueResources.get(queueName),
-            maxQueueResources.get(queueName))) {
-      LOG.warn(String.format("Queue %s has max resources %d less than min resources %d",
-          queueName, maxQueueResources.get(queueName), minQueueResources.get(queueName)));
-    }
-  }
-
-  /**
-   * Get the minimum resource allocation for the given queue.
-   * @return the cap set on this queue, or 0 if not set.
-   */
-  public Resource getMinResources(String queue) {
-    Resource minQueueResource = info.minQueueResources.get(queue);
-    if (minQueueResource != null) {
-      return minQueueResource;
-    } else {
-      return Resources.createResource(0);
-    }
-  }
-
-  /**
-   * Get the maximum resource allocation for the given queue.
-   * @return the cap set on this queue, or Integer.MAX_VALUE if not set.
-   */
-
-  public Resource getMaxResources(String queueName) {
-    Resource maxQueueResource = info.maxQueueResources.get(queueName);
-    if (maxQueueResource != null) {
-      return maxQueueResource;
-    } else {
-      return Resources.createResource(Integer.MAX_VALUE, Integer.MAX_VALUE);
-    }
-  }
-
   /**
    * Get a collection of all leaf queues
    */
@@ -567,141 +213,27 @@ public class QueueManager {
   public Collection<FSQueue> getQueues() {
     return queues.values();
   }
-
-  public int getUserMaxApps(String user) {
-    // save current info in case it gets changed under us
-    QueueManagerInfo info = this.info;
-    if (info.userMaxApps.containsKey(user)) {
-      return info.userMaxApps.get(user);
-    } else {
-      return info.userMaxAppsDefault;
-    }
-  }
-
-  public int getQueueMaxApps(String queue) {
-    // save current info in case it gets changed under us
-    QueueManagerInfo info = this.info;
-    if (info.queueMaxApps.containsKey(queue)) {
-      return info.queueMaxApps.get(queue);
-    } else {
-      return info.queueMaxAppsDefault;
-    }
-  }
   
-  public ResourceWeights getQueueWeight(String queue) {
-    ResourceWeights weight = info.queueWeights.get(queue);
-    if (weight != null) {
-      return weight;
-    } else {
-      return ResourceWeights.NEUTRAL;
+  public void updateAllocationConfiguration(AllocationConfiguration queueConf) {
+    // Make sure all queues exist
+    for (String name : queueConf.getQueueNames()) {
+      getLeafQueue(name, true);
     }
-  }
-
-  /**
-   * Get a queue's min share preemption timeout, in milliseconds. This is the
-   * time after which jobs in the queue may kill other queues' tasks if they
-   * are below their min share.
-   */
-  public long getMinSharePreemptionTimeout(String queueName) {
-    // save current info in case it gets changed under us
-    QueueManagerInfo info = this.info;
-    if (info.minSharePreemptionTimeouts.containsKey(queueName)) {
-      return info.minSharePreemptionTimeouts.get(queueName);
-    }
-    return info.defaultMinSharePreemptionTimeout;
-  }
-  
-  /**
-   * Get the fair share preemption, in milliseconds. This is the time
-   * after which any job may kill other jobs' tasks if it is below half
-   * its fair share.
-   */
-  public long getFairSharePreemptionTimeout() {
-    return info.fairSharePreemptionTimeout;
-  }
-
-  /**
-   * Get the ACLs associated with this queue. If a given ACL is not explicitly
-   * configured, include the default value for that ACL.  The default for the
-   * root queue is everybody ("*") and the default for all other queues is
-   * nobody ("")
-   */
-  public AccessControlList getQueueAcl(String queue, QueueACL operation) {
-    Map<QueueACL, AccessControlList> queueAcls = info.queueAcls.get(queue);
-    if (queueAcls == null || !queueAcls.containsKey(operation)) {
-      return (queue.equals(ROOT_QUEUE)) ? EVERYBODY_ACL : NOBODY_ACL;
-    }
-    return queueAcls.get(operation);
-  }
-  
-  static class QueueManagerInfo {
-    // Minimum resource allocation for each queue
-    public final Map<String, Resource> minQueueResources;
-    // Maximum amount of resources per queue
-    public final Map<String, Resource> maxQueueResources;
-    // Sharing weights for each queue
-    public final Map<String, ResourceWeights> queueWeights;
     
-    // Max concurrent running applications for each queue and for each user; in addition,
-    // for users that have no max specified, we use the userMaxJobsDefault.
-    public final Map<String, Integer> queueMaxApps;
-    public final Map<String, Integer> userMaxApps;
-    public final int userMaxAppsDefault;
-    public final int queueMaxAppsDefault;
-
-    // ACL's for each queue. Only specifies non-default ACL's from configuration.
-    public final Map<String, Map<QueueACL, AccessControlList>> queueAcls;
-
-    // Min share preemption timeout for each queue in seconds. If a job in the queue
-    // waits this long without receiving its guaranteed share, it is allowed to
-    // preempt other jobs' tasks.
-    public final Map<String, Long> minSharePreemptionTimeouts;
-
-    // Default min share preemption timeout for queues where it is not set
-    // explicitly.
-    public final long defaultMinSharePreemptionTimeout;
-
-    // Preemption timeout for jobs below fair share in seconds. If a job remains
-    // below half its fair share for this long, it is allowed to preempt tasks.
-    public final long fairSharePreemptionTimeout;
-
-    public final SchedulingPolicy defaultSchedulingPolicy;
-    
-    public QueueManagerInfo(Map<String, Resource> minQueueResources, 
-        Map<String, Resource> maxQueueResources, 
-        Map<String, Integer> queueMaxApps, Map<String, Integer> userMaxApps,
-        Map<String, ResourceWeights> queueWeights, int userMaxAppsDefault,
-        int queueMaxAppsDefault, SchedulingPolicy defaultSchedulingPolicy, 
-        Map<String, Long> minSharePreemptionTimeouts, 
-        Map<String, Map<QueueACL, AccessControlList>> queueAcls,
-        long fairSharePreemptionTimeout, long defaultMinSharePreemptionTimeout) {
-      this.minQueueResources = minQueueResources;
-      this.maxQueueResources = maxQueueResources;
-      this.queueMaxApps = queueMaxApps;
-      this.userMaxApps = userMaxApps;
-      this.queueWeights = queueWeights;
-      this.userMaxAppsDefault = userMaxAppsDefault;
-      this.queueMaxAppsDefault = queueMaxAppsDefault;
-      this.defaultSchedulingPolicy = defaultSchedulingPolicy;
-      this.minSharePreemptionTimeouts = minSharePreemptionTimeouts;
-      this.queueAcls = queueAcls;
-      this.fairSharePreemptionTimeout = fairSharePreemptionTimeout;
-      this.defaultMinSharePreemptionTimeout = defaultMinSharePreemptionTimeout;
-    }
-    
-    public QueueManagerInfo() {
-      minQueueResources = new HashMap<String, Resource>();
-      maxQueueResources = new HashMap<String, Resource>();
-      queueWeights = new HashMap<String, ResourceWeights>();
-      queueMaxApps = new HashMap<String, Integer>();
-      userMaxApps = new HashMap<String, Integer>();
-      userMaxAppsDefault = Integer.MAX_VALUE;
-      queueMaxAppsDefault = Integer.MAX_VALUE;
-      queueAcls = new HashMap<String, Map<QueueACL, AccessControlList>>();
-      minSharePreemptionTimeouts = new HashMap<String, Long>();
-      defaultMinSharePreemptionTimeout = Long.MAX_VALUE;
-      fairSharePreemptionTimeout = Long.MAX_VALUE;
-      defaultSchedulingPolicy = SchedulingPolicy.getDefault();
+    for (FSQueue queue : queues.values()) {
+      // Update queue metrics
+      FSQueueMetrics queueMetrics = queue.getMetrics();
+      queueMetrics.setMinShare(queue.getMinShare());
+      queueMetrics.setMaxShare(queue.getMaxShare());
+      // Set scheduling policies
+      try {
+        SchedulingPolicy policy = queueConf.getSchedulingPolicy(queue.getName());
+        policy.initialize(scheduler.getClusterCapacity());
+        queue.setPolicy(policy);
+      } catch (AllocationConfigurationException ex) {
+        LOG.warn("Cannot apply configured scheduling policy to queue "
+            + queue.getName(), ex);
+      }
     }
   }
 }

+ 32 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueuePlacementPolicy.java

@@ -94,6 +94,34 @@ public class QueuePlacementPolicy {
     return new QueuePlacementPolicy(rules, configuredQueues, conf);
   }
   
+  /**
+   * Build a simple queue placement policy from the allow-undeclared-pools and
+   * user-as-default-queue configuration options.
+   */
+  public static QueuePlacementPolicy fromConfiguration(Configuration conf,
+      Set<String> configuredQueues) {
+    boolean create = conf.getBoolean(
+        FairSchedulerConfiguration.ALLOW_UNDECLARED_POOLS,
+        FairSchedulerConfiguration.DEFAULT_ALLOW_UNDECLARED_POOLS);
+    boolean userAsDefaultQueue = conf.getBoolean(
+        FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE,
+        FairSchedulerConfiguration.DEFAULT_USER_AS_DEFAULT_QUEUE);
+    List<QueuePlacementRule> rules = new ArrayList<QueuePlacementRule>();
+    rules.add(new QueuePlacementRule.Specified().initialize(create, null));
+    if (userAsDefaultQueue) {
+      rules.add(new QueuePlacementRule.User().initialize(create, null));
+    }
+    if (!userAsDefaultQueue || !create) {
+      rules.add(new QueuePlacementRule.Default().initialize(true, null));
+    }
+    try {
+      return new QueuePlacementPolicy(rules, configuredQueues, conf);
+    } catch (AllocationConfigurationException ex) {
+      throw new RuntimeException("Should never hit exception when loading" +
+      		"placement policy from conf", ex);
+    }
+  }
+
   /**
    * Applies this rule to an app with the given requested queue and user/group
    * information.
@@ -120,4 +148,8 @@ public class QueuePlacementPolicy {
     throw new IllegalStateException("Should have applied a rule before " +
     		"reaching here");
   }
+  
+  public List<QueuePlacementRule> getRules() {
+    return rules;
+  }
 }

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

@@ -35,7 +35,7 @@ public abstract class SchedulingPolicy {
   private static final ConcurrentHashMap<Class<? extends SchedulingPolicy>, SchedulingPolicy> instances =
       new ConcurrentHashMap<Class<? extends SchedulingPolicy>, SchedulingPolicy>();
 
-  private static SchedulingPolicy DEFAULT_POLICY =
+  public static final SchedulingPolicy DEFAULT_POLICY =
       getInstance(FairSharePolicy.class);
   
   public static final byte DEPTH_LEAF = (byte) 1;
@@ -44,15 +44,6 @@ public abstract class SchedulingPolicy {
   public static final byte DEPTH_PARENT = (byte) 6; // Root and Intermediate
   public static final byte DEPTH_ANY = (byte) 7;
 
-  public static SchedulingPolicy getDefault() {
-    return DEFAULT_POLICY;
-  }
-
-  public static void setDefault(String className)
-      throws AllocationConfigurationException {
-    DEFAULT_POLICY = parse(className);
-  }
-
   /**
    * Returns a {@link SchedulingPolicy} instance corresponding to the passed clazz
    */

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

@@ -29,10 +29,10 @@ import javax.xml.bind.annotation.XmlSeeAlso;
 import javax.xml.bind.annotation.XmlTransient;
 
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.AllocationConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSLeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueueManager;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 @XmlRootElement
@@ -65,7 +65,7 @@ public class FairSchedulerQueueInfo {
   }
   
   public FairSchedulerQueueInfo(FSQueue queue, FairScheduler scheduler) {
-    QueueManager manager = scheduler.getQueueManager();
+    AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
     
     queueName = queue.getName();
     schedulingPolicy = queue.getPolicy().getName();
@@ -87,7 +87,7 @@ public class FairSchedulerQueueInfo {
     fractionMemMinShare = (float)minResources.getMemory() / clusterResources.getMemory();
     fractionMemMaxShare = (float)maxResources.getMemory() / clusterResources.getMemory();
     
-    maxApps = manager.getQueueMaxApps(queueName);
+    maxApps = allocConf.getQueueMaxApps(queueName);
     
     Collection<FSQueue> children = queue.getChildQueues();
     childQueues = new ArrayList<FairSchedulerQueueInfo>();

+ 432 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java

@@ -0,0 +1,432 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
+
+import static junit.framework.Assert.*;
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.PrintWriter;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.QueueACL;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FairSharePolicy;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Test;
+
+public class TestAllocationFileLoaderService {
+  
+  final static String TEST_DIR = new File(System.getProperty("test.build.data",
+      "/tmp")).getAbsolutePath();
+
+  final static String ALLOC_FILE = new File(TEST_DIR,
+      "test-queues").getAbsolutePath();
+  
+  private class MockClock implements Clock {
+    private long time = 0;
+    @Override
+    public long getTime() {
+      return time;
+    }
+
+    public void tick(long ms) {
+      time += ms;
+    }
+  }
+  
+  @Test
+  public void testGetAllocationFileFromClasspath() {
+    Configuration conf = new Configuration();
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE,
+        "test-fair-scheduler.xml");
+    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
+    File allocationFile = allocLoader.getAllocationFile(conf);
+    assertEquals("test-fair-scheduler.xml", allocationFile.getName());
+    assertTrue(allocationFile.exists());
+  }
+  
+  @Test (timeout = 10000)
+  public void testReload() throws Exception {
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("  <queue name=\"queueA\">");
+    out.println("    <maxRunningApps>1</maxRunningApps>");
+    out.println("  </queue>");
+    out.println("  <queue name=\"queueB\" />");
+    out.println("  <queuePlacementPolicy>");
+    out.println("    <default />");
+    out.println("  </queuePlacementPolicy>");
+    out.println("</allocations>");
+    out.close();
+    
+    MockClock clock = new MockClock();
+    Configuration conf = new Configuration();
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+
+    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(
+        clock);
+    allocLoader.reloadIntervalMs = 5;
+    allocLoader.init(conf);
+    ReloadListener confHolder = new ReloadListener();
+    allocLoader.setReloadListener(confHolder);
+    allocLoader.reloadAllocations();
+    AllocationConfiguration allocConf = confHolder.allocConf;
+    
+    // Verify conf
+    QueuePlacementPolicy policy = allocConf.getPlacementPolicy();
+    List<QueuePlacementRule> rules = policy.getRules();
+    assertEquals(1, rules.size());
+    assertEquals(QueuePlacementRule.Default.class, rules.get(0).getClass());
+    assertEquals(1, allocConf.getQueueMaxApps("root.queueA"));
+    assertEquals(2, allocConf.getQueueNames().size());
+    assertTrue(allocConf.getQueueNames().contains("root.queueA"));
+    assertTrue(allocConf.getQueueNames().contains("root.queueB"));
+    
+    confHolder.allocConf = null;
+    
+    // Modify file and advance the clock
+    out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("  <queue name=\"queueB\">");
+    out.println("    <maxRunningApps>3</maxRunningApps>");
+    out.println("  </queue>");
+    out.println("  <queuePlacementPolicy>");
+    out.println("    <specified />");
+    out.println("    <default />");
+    out.println("  </queuePlacementPolicy>");
+    out.println("</allocations>");
+    out.close();
+    
+    clock.tick(System.currentTimeMillis()
+        + AllocationFileLoaderService.ALLOC_RELOAD_WAIT_MS + 10000);
+    allocLoader.start();
+    
+    while (confHolder.allocConf == null) {
+      Thread.sleep(20);
+    }
+    
+    // Verify conf
+    allocConf = confHolder.allocConf;
+    policy = allocConf.getPlacementPolicy();
+    rules = policy.getRules();
+    assertEquals(2, rules.size());
+    assertEquals(QueuePlacementRule.Specified.class, rules.get(0).getClass());
+    assertEquals(QueuePlacementRule.Default.class, rules.get(1).getClass());
+    assertEquals(3, allocConf.getQueueMaxApps("root.queueB"));
+    assertEquals(1, allocConf.getQueueNames().size());
+    assertTrue(allocConf.getQueueNames().contains("root.queueB"));
+  }
+  
+  @Test
+  public void testAllocationFileParsing() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
+
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    // Give queue A a minimum of 1024 M
+    out.println("<queue name=\"queueA\">");
+    out.println("<minResources>1024mb,0vcores</minResources>");
+    out.println("</queue>");
+    // Give queue B a minimum of 2048 M
+    out.println("<queue name=\"queueB\">");
+    out.println("<minResources>2048mb,0vcores</minResources>");
+    out.println("<aclAdministerApps>alice,bob admins</aclAdministerApps>");
+    out.println("<schedulingPolicy>fair</schedulingPolicy>");
+    out.println("</queue>");
+    // Give queue C no minimum
+    out.println("<queue name=\"queueC\">");
+    out.println("<aclSubmitApps>alice,bob admins</aclSubmitApps>");
+    out.println("</queue>");
+    // Give queue D a limit of 3 running apps
+    out.println("<queue name=\"queueD\">");
+    out.println("<maxRunningApps>3</maxRunningApps>");
+    out.println("</queue>");
+    // Give queue E a preemption timeout of one minute
+    out.println("<queue name=\"queueE\">");
+    out.println("<minSharePreemptionTimeout>60</minSharePreemptionTimeout>");
+    out.println("</queue>");
+    // Set default limit of apps per queue to 15
+    out.println("<queueMaxAppsDefault>15</queueMaxAppsDefault>");
+    // Set default limit of apps per user to 5
+    out.println("<userMaxAppsDefault>5</userMaxAppsDefault>");
+    // Give user1 a limit of 10 jobs
+    out.println("<user name=\"user1\">");
+    out.println("<maxRunningApps>10</maxRunningApps>");
+    out.println("</user>");
+    // Set default min share preemption timeout to 2 minutes
+    out.println("<defaultMinSharePreemptionTimeout>120"
+        + "</defaultMinSharePreemptionTimeout>");
+    // Set fair share preemption timeout to 5 minutes
+    out.println("<fairSharePreemptionTimeout>300</fairSharePreemptionTimeout>");
+    // Set default scheduling policy to DRF
+    out.println("<defaultQueueSchedulingPolicy>drf</defaultQueueSchedulingPolicy>");
+    out.println("</allocations>");
+    out.close();
+    
+    allocLoader.init(conf);
+    ReloadListener confHolder = new ReloadListener();
+    allocLoader.setReloadListener(confHolder);
+    allocLoader.reloadAllocations();
+    AllocationConfiguration queueConf = confHolder.allocConf;
+    
+    assertEquals(5, queueConf.getQueueNames().size());
+    assertEquals(Resources.createResource(0),
+        queueConf.getMinResources("root." + YarnConfiguration.DEFAULT_QUEUE_NAME));
+    assertEquals(Resources.createResource(0),
+        queueConf.getMinResources("root." + YarnConfiguration.DEFAULT_QUEUE_NAME));
+
+    assertEquals(Resources.createResource(1024, 0),
+        queueConf.getMinResources("root.queueA"));
+    assertEquals(Resources.createResource(2048, 0),
+        queueConf.getMinResources("root.queueB"));
+    assertEquals(Resources.createResource(0),
+        queueConf.getMinResources("root.queueC"));
+    assertEquals(Resources.createResource(0),
+        queueConf.getMinResources("root.queueD"));
+    assertEquals(Resources.createResource(0),
+        queueConf.getMinResources("root.queueE"));
+
+    assertEquals(15, queueConf.getQueueMaxApps("root." + YarnConfiguration.DEFAULT_QUEUE_NAME));
+    assertEquals(15, queueConf.getQueueMaxApps("root.queueA"));
+    assertEquals(15, queueConf.getQueueMaxApps("root.queueB"));
+    assertEquals(15, queueConf.getQueueMaxApps("root.queueC"));
+    assertEquals(3, queueConf.getQueueMaxApps("root.queueD"));
+    assertEquals(15, queueConf.getQueueMaxApps("root.queueE"));
+    assertEquals(10, queueConf.getUserMaxApps("user1"));
+    assertEquals(5, queueConf.getUserMaxApps("user2"));
+
+    // Root should get * ACL
+    assertEquals("*", queueConf.getQueueAcl("root",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals("*", queueConf.getQueueAcl("root",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
+
+    // Unspecified queues should get default ACL
+    assertEquals(" ", queueConf.getQueueAcl("root.queueA",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals(" ", queueConf.getQueueAcl("root.queueA",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
+
+    // Queue B ACL
+    assertEquals("alice,bob admins", queueConf.getQueueAcl("root.queueB",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
+
+    // Queue C ACL
+    assertEquals("alice,bob admins", queueConf.getQueueAcl("root.queueC",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
+
+    assertEquals(120000, queueConf.getMinSharePreemptionTimeout("root." + 
+        YarnConfiguration.DEFAULT_QUEUE_NAME));
+    assertEquals(120000, queueConf.getMinSharePreemptionTimeout("root.queueA"));
+    assertEquals(120000, queueConf.getMinSharePreemptionTimeout("root.queueB"));
+    assertEquals(120000, queueConf.getMinSharePreemptionTimeout("root.queueC"));
+    assertEquals(120000, queueConf.getMinSharePreemptionTimeout("root.queueD"));
+    assertEquals(120000, queueConf.getMinSharePreemptionTimeout("root.queueA"));
+    assertEquals(60000, queueConf.getMinSharePreemptionTimeout("root.queueE"));
+    assertEquals(300000, queueConf.getFairSharePreemptionTimeout());
+    
+    // Verify existing queues have default scheduling policy
+    assertEquals(DominantResourceFairnessPolicy.NAME,
+        queueConf.getSchedulingPolicy("root").getName());
+    assertEquals(DominantResourceFairnessPolicy.NAME,
+        queueConf.getSchedulingPolicy("root.queueA").getName());
+    // Verify default is overriden if specified explicitly
+    assertEquals(FairSharePolicy.NAME,
+        queueConf.getSchedulingPolicy("root.queueB").getName());
+    // Verify new queue gets default scheduling policy
+    assertEquals(DominantResourceFairnessPolicy.NAME,
+        queueConf.getSchedulingPolicy("root.newqueue").getName());
+  }
+  
+  @Test
+  public void testBackwardsCompatibleAllocationFileParsing() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
+
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    // Give queue A a minimum of 1024 M
+    out.println("<pool name=\"queueA\">");
+    out.println("<minResources>1024mb,0vcores</minResources>");
+    out.println("</pool>");
+    // Give queue B a minimum of 2048 M
+    out.println("<pool name=\"queueB\">");
+    out.println("<minResources>2048mb,0vcores</minResources>");
+    out.println("<aclAdministerApps>alice,bob admins</aclAdministerApps>");
+    out.println("</pool>");
+    // Give queue C no minimum
+    out.println("<pool name=\"queueC\">");
+    out.println("<aclSubmitApps>alice,bob admins</aclSubmitApps>");
+    out.println("</pool>");
+    // Give queue D a limit of 3 running apps
+    out.println("<pool name=\"queueD\">");
+    out.println("<maxRunningApps>3</maxRunningApps>");
+    out.println("</pool>");
+    // Give queue E a preemption timeout of one minute
+    out.println("<pool name=\"queueE\">");
+    out.println("<minSharePreemptionTimeout>60</minSharePreemptionTimeout>");
+    out.println("</pool>");
+    // Set default limit of apps per queue to 15
+    out.println("<queueMaxAppsDefault>15</queueMaxAppsDefault>");
+    // Set default limit of apps per user to 5
+    out.println("<userMaxAppsDefault>5</userMaxAppsDefault>");
+    // Give user1 a limit of 10 jobs
+    out.println("<user name=\"user1\">");
+    out.println("<maxRunningApps>10</maxRunningApps>");
+    out.println("</user>");
+    // Set default min share preemption timeout to 2 minutes
+    out.println("<defaultMinSharePreemptionTimeout>120"
+        + "</defaultMinSharePreemptionTimeout>");
+    // Set fair share preemption timeout to 5 minutes
+    out.println("<fairSharePreemptionTimeout>300</fairSharePreemptionTimeout>");
+    out.println("</allocations>");
+    out.close();
+    
+    allocLoader.init(conf);
+    ReloadListener confHolder = new ReloadListener();
+    allocLoader.setReloadListener(confHolder);
+    allocLoader.reloadAllocations();
+    AllocationConfiguration queueConf = confHolder.allocConf;
+
+    assertEquals(5, queueConf.getQueueNames().size());
+    assertEquals(Resources.createResource(0),
+        queueConf.getMinResources("root." + YarnConfiguration.DEFAULT_QUEUE_NAME));
+    assertEquals(Resources.createResource(0),
+        queueConf.getMinResources("root." + YarnConfiguration.DEFAULT_QUEUE_NAME));
+
+    assertEquals(Resources.createResource(1024, 0),
+        queueConf.getMinResources("root.queueA"));
+    assertEquals(Resources.createResource(2048, 0),
+        queueConf.getMinResources("root.queueB"));
+    assertEquals(Resources.createResource(0),
+        queueConf.getMinResources("root.queueC"));
+    assertEquals(Resources.createResource(0),
+        queueConf.getMinResources("root.queueD"));
+    assertEquals(Resources.createResource(0),
+        queueConf.getMinResources("root.queueE"));
+
+    assertEquals(15, queueConf.getQueueMaxApps("root." + YarnConfiguration.DEFAULT_QUEUE_NAME));
+    assertEquals(15, queueConf.getQueueMaxApps("root.queueA"));
+    assertEquals(15, queueConf.getQueueMaxApps("root.queueB"));
+    assertEquals(15, queueConf.getQueueMaxApps("root.queueC"));
+    assertEquals(3, queueConf.getQueueMaxApps("root.queueD"));
+    assertEquals(15, queueConf.getQueueMaxApps("root.queueE"));
+    assertEquals(10, queueConf.getUserMaxApps("user1"));
+    assertEquals(5, queueConf.getUserMaxApps("user2"));
+
+    // Unspecified queues should get default ACL
+    assertEquals(" ", queueConf.getQueueAcl("root.queueA",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals(" ", queueConf.getQueueAcl("root.queueA",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
+
+    // Queue B ACL
+    assertEquals("alice,bob admins", queueConf.getQueueAcl("root.queueB",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
+
+    // Queue C ACL
+    assertEquals("alice,bob admins", queueConf.getQueueAcl("root.queueC",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
+
+
+    assertEquals(120000, queueConf.getMinSharePreemptionTimeout("root." +
+        YarnConfiguration.DEFAULT_QUEUE_NAME));
+    assertEquals(120000, queueConf.getMinSharePreemptionTimeout("root.queueA"));
+    assertEquals(120000, queueConf.getMinSharePreemptionTimeout("root.queueB"));
+    assertEquals(120000, queueConf.getMinSharePreemptionTimeout("root.queueC"));
+    assertEquals(120000, queueConf.getMinSharePreemptionTimeout("root.queueD"));
+    assertEquals(120000, queueConf.getMinSharePreemptionTimeout("root.queueA"));
+    assertEquals(60000, queueConf.getMinSharePreemptionTimeout("root.queueE"));
+    assertEquals(300000, queueConf.getFairSharePreemptionTimeout());
+  }
+  
+  @Test
+  public void testSimplePlacementPolicyFromConf() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+    conf.setBoolean(FairSchedulerConfiguration.ALLOW_UNDECLARED_POOLS, false);
+    conf.setBoolean(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, false);
+    
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("</allocations>");
+    out.close();
+    
+    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
+    allocLoader.init(conf);
+    ReloadListener confHolder = new ReloadListener();
+    allocLoader.setReloadListener(confHolder);
+    allocLoader.reloadAllocations();
+    AllocationConfiguration allocConf = confHolder.allocConf;
+    
+    QueuePlacementPolicy placementPolicy = allocConf.getPlacementPolicy();
+    List<QueuePlacementRule> rules = placementPolicy.getRules();
+    assertEquals(2, rules.size());
+    assertEquals(QueuePlacementRule.Specified.class, rules.get(0).getClass());
+    assertEquals(false, rules.get(0).create);
+    assertEquals(QueuePlacementRule.Default.class, rules.get(1).getClass());
+  }
+  
+  /**
+   * Verify that you can't place queues at the same level as the root queue in
+   * the allocations file.
+   */
+  @Test (expected = AllocationConfigurationException.class)
+  public void testQueueAlongsideRoot() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"root\">");
+    out.println("</queue>");
+    out.println("<queue name=\"other\">");
+    out.println("</queue>");
+    out.println("</allocations>");
+    out.close();
+    
+    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
+    allocLoader.init(conf);
+    ReloadListener confHolder = new ReloadListener();
+    allocLoader.setReloadListener(confHolder);
+    allocLoader.reloadAllocations();
+  }
+  
+  private class ReloadListener implements AllocationFileLoaderService.Listener {
+    public AllocationConfiguration allocConf;
+    
+    @Override
+    public void onReload(AllocationConfiguration info) {
+      allocConf = info;
+    }
+  }
+}

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java

@@ -51,11 +51,11 @@ public class TestFSLeafQueue {
     scheduler.reinitialize(conf, resourceManager.getRMContext());
     
     String queueName = "root.queue1";
-    QueueManager mockMgr = mock(QueueManager.class);
-    when(mockMgr.getMaxResources(queueName)).thenReturn(maxResource);
-    when(mockMgr.getMinResources(queueName)).thenReturn(Resources.none());
+    scheduler.allocConf = mock(AllocationConfiguration.class);
+    when(scheduler.allocConf.getMaxResources(queueName)).thenReturn(maxResource);
+    when(scheduler.allocConf.getMinResources(queueName)).thenReturn(Resources.none());
 
-    schedulable = new FSLeafQueue(queueName, mockMgr, scheduler, null);
+    schedulable = new FSLeafQueue(queueName, scheduler, null);
   }
 
   @Test

+ 113 - 342
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

@@ -55,7 +55,6 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -86,7 +85,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSc
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FairSharePolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Clock;
@@ -121,6 +119,7 @@ public class TestFairScheduler {
 
   private FairScheduler scheduler;
   private ResourceManager resourceManager;
+  private Configuration conf;
   private static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
 
   private int APP_ID = 1; // Incrementing counter for schedling apps
@@ -130,7 +129,7 @@ public class TestFairScheduler {
   @Before
   public void setUp() throws IOException {
     scheduler = new FairScheduler();
-    Configuration conf = createConfiguration();
+    conf = createConfiguration();
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 0);
     conf.setInt(FairSchedulerConfiguration.RM_SCHEDULER_INCREMENT_ALLOCATION_MB,
       1024);
@@ -145,7 +144,6 @@ public class TestFairScheduler {
     ((AsyncDispatcher)resourceManager.getRMContext().getDispatcher()).start();
     resourceManager.getRMContext().getStateStore().start();
 
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
     // to initialize the master key
     resourceManager.getRMContainerTokenSecretManager().rollMasterKey();
   }
@@ -291,7 +289,6 @@ public class TestFairScheduler {
 
   @Test(timeout=2000)
   public void testLoadConfigurationOnInitialize() throws IOException {
-    Configuration conf = createConfiguration();
     conf.setBoolean(FairSchedulerConfiguration.ASSIGN_MULTIPLE, true);
     conf.setInt(FairSchedulerConfiguration.MAX_ASSIGN, 3);
     conf.setBoolean(FairSchedulerConfiguration.SIZE_BASED_WEIGHT, true);
@@ -362,6 +359,8 @@ public class TestFairScheduler {
   
   @Test
   public void testAggregateCapacityTracking() throws Exception {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     // Add a node
     RMNode node1 =
         MockNodes
@@ -384,7 +383,9 @@ public class TestFairScheduler {
   }
 
   @Test
-  public void testSimpleFairShareCalculation() {
+  public void testSimpleFairShareCalculation() throws IOException {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     // Add one big node (only care about aggregate capacity)
     RMNode node1 =
         MockNodes.newNodeInfo(1, Resources.createResource(10 * 1024), 1,
@@ -409,7 +410,9 @@ public class TestFairScheduler {
   }
   
   @Test
-  public void testSimpleHierarchicalFairShareCalculation() {
+  public void testSimpleHierarchicalFairShareCalculation() throws IOException {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     // Add one big node (only care about aggregate capacity)
     int capacity = 10 * 24;
     RMNode node1 =
@@ -440,7 +443,9 @@ public class TestFairScheduler {
   }
 
   @Test
-  public void testHierarchicalQueuesSimilarParents() {
+  public void testHierarchicalQueuesSimilarParents() throws IOException {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     QueueManager queueManager = scheduler.getQueueManager();
     FSLeafQueue leafQueue = queueManager.getLeafQueue("parent.child", true);
     Assert.assertEquals(2, queueManager.getLeafQueues().size());
@@ -462,8 +467,9 @@ public class TestFairScheduler {
   }
 
   @Test
-  public void testSchedulerRootQueueMetrics() throws InterruptedException {
-	  
+  public void testSchedulerRootQueueMetrics() throws Exception {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     // Add a node
     RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024));
     NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
@@ -500,7 +506,9 @@ public class TestFairScheduler {
   }
 
   @Test (timeout = 5000)
-  public void testSimpleContainerAllocation() {
+  public void testSimpleContainerAllocation() throws IOException {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     // Add a node
     RMNode node1 =
         MockNodes
@@ -546,7 +554,9 @@ public class TestFairScheduler {
   }
 
   @Test (timeout = 5000)
-  public void testSimpleContainerReservation() throws InterruptedException {
+  public void testSimpleContainerReservation() throws Exception {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     // Add a node
     RMNode node1 =
         MockNodes
@@ -598,7 +608,6 @@ public class TestFairScheduler {
 
   @Test
   public void testUserAsDefaultQueue() throws Exception {
-    Configuration conf = createConfiguration();
     conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "true");
     scheduler.reinitialize(conf, resourceManager.getRMContext());
     RMContext rmContext = resourceManager.getRMContext();
@@ -617,14 +626,24 @@ public class TestFairScheduler {
     assertEquals(0, scheduler.getQueueManager().getLeafQueue("default", true)
         .getRunnableAppSchedulables().size());
     assertEquals("root.user1", rmApp.getQueue());
-
+  }
+  
+  @Test
+  public void testNotUserAsDefaultQueue() throws Exception {
     conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "false");
     scheduler.reinitialize(conf, resourceManager.getRMContext());
-    scheduler.getQueueManager().initialize();
+    RMContext rmContext = resourceManager.getRMContext();
+    Map<ApplicationId, RMApp> appsMap = rmContext.getRMApps();
+    ApplicationAttemptId appAttemptId = createAppAttemptId(1, 1);
+    RMApp rmApp = new RMAppImpl(appAttemptId.getApplicationId(), rmContext, conf,
+        null, null, null, ApplicationSubmissionContext.newInstance(null, null,
+            null, null, null, false, false, 0, null, null), null, null, 0, null);
+    appsMap.put(appAttemptId.getApplicationId(), rmApp);
+    
     AppAddedSchedulerEvent appAddedEvent2 = new AppAddedSchedulerEvent(
-        createAppAttemptId(2, 1), "default", "user2");
+        appAttemptId, "default", "user2");
     scheduler.handle(appAddedEvent2);
-    assertEquals(1, scheduler.getQueueManager().getLeafQueue("user1", true)
+    assertEquals(0, scheduler.getQueueManager().getLeafQueue("user1", true)
         .getRunnableAppSchedulables().size());
     assertEquals(1, scheduler.getQueueManager().getLeafQueue("default", true)
         .getRunnableAppSchedulables().size());
@@ -634,7 +653,7 @@ public class TestFairScheduler {
 
   @Test
   public void testEmptyQueueName() throws Exception {
-    Configuration conf = createConfiguration();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     // only default queue
     assertEquals(1, scheduler.getQueueManager().getLeafQueues().size());
@@ -653,7 +672,6 @@ public class TestFairScheduler {
 
   @Test
   public void testAssignToQueue() throws Exception {
-    Configuration conf = createConfiguration();
     conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "true");
     scheduler.reinitialize(conf, resourceManager.getRMContext());
     
@@ -672,9 +690,10 @@ public class TestFairScheduler {
   
   @Test
   public void testQueuePlacementWithPolicy() throws Exception {
-    Configuration conf = createConfiguration();
     conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
         SimpleGroupsMapping.class, GroupMappingServiceProvider.class);
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     ApplicationAttemptId appId;
     Map<ApplicationAttemptId, FSSchedulerApp> apps = scheduler.applications;
 
@@ -684,10 +703,10 @@ public class TestFairScheduler {
     rules.add(new QueuePlacementRule.PrimaryGroup().initialize(false, null));
     rules.add(new QueuePlacementRule.SecondaryGroupExistingQueue().initialize(false, null));
     rules.add(new QueuePlacementRule.Default().initialize(true, null));
-    Set<String> queues = Sets.newHashSet("root.user1", "root.user3group", 
+    Set<String> queues = Sets.newHashSet("root.user1", "root.user3group",
         "root.user4subgroup1", "root.user4subgroup2" , "root.user5subgroup2");
-    scheduler.getQueueManager().placementPolicy = new QueuePlacementPolicy(
-        rules, queues, conf);
+    scheduler.getAllocationConfiguration().placementPolicy =
+        new QueuePlacementPolicy(rules, queues, conf);
     appId = createSchedulingRequest(1024, "somequeue", "user1");
     assertEquals("root.somequeue", apps.get(appId).getQueueName());
     appId = createSchedulingRequest(1024, "default", "user1");
@@ -706,8 +725,8 @@ public class TestFairScheduler {
     rules.add(new QueuePlacementRule.User().initialize(false, null));
     rules.add(new QueuePlacementRule.Specified().initialize(true, null));
     rules.add(new QueuePlacementRule.Default().initialize(true, null));
-    scheduler.getQueueManager().placementPolicy = new QueuePlacementPolicy(
-        rules, queues, conf);
+    scheduler.getAllocationConfiguration().placementPolicy =
+        new QueuePlacementPolicy(rules, queues, conf);
     appId = createSchedulingRequest(1024, "somequeue", "user1");
     assertEquals("root.user1", apps.get(appId).getQueueName());
     appId = createSchedulingRequest(1024, "somequeue", "otheruser");
@@ -718,9 +737,7 @@ public class TestFairScheduler {
 
   @Test
   public void testFairShareWithMinAlloc() throws Exception {
-    Configuration conf = createConfiguration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
@@ -733,9 +750,8 @@ public class TestFairScheduler {
     out.println("</queue>");
     out.println("</allocations>");
     out.close();
-
-    QueueManager queueManager = scheduler.getQueueManager();
-    queueManager.initialize();
+    
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     // Add one big node (only care about aggregate capacity)
     RMNode node1 =
@@ -767,6 +783,8 @@ public class TestFairScheduler {
    */
   @Test
   public void testQueueDemandCalculation() throws Exception {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     ApplicationAttemptId id11 = createAppAttemptId(1, 1);
     scheduler.addApplication(id11, "root.queue1", "user1");
     ApplicationAttemptId id21 = createAppAttemptId(2, 1);
@@ -812,6 +830,8 @@ public class TestFairScheduler {
 
   @Test
   public void testAppAdditionAndRemoval() throws Exception {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     AppAddedSchedulerEvent appAddedEvent1 = new AppAddedSchedulerEvent(
         createAppAttemptId(1, 1), "default", "user1");
     scheduler.handle(appAddedEvent1);
@@ -834,133 +854,10 @@ public class TestFairScheduler {
         .getRunnableAppSchedulables().size());
   }
 
-  @Test
-  public void testAllocationFileParsing() throws Exception {
-    Configuration conf = createConfiguration();
-    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
-
-    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
-    out.println("<?xml version=\"1.0\"?>");
-    out.println("<allocations>");
-    // Give queue A a minimum of 1024 M
-    out.println("<queue name=\"queueA\">");
-    out.println("<minResources>1024mb,0vcores</minResources>");
-    out.println("</queue>");
-    // Give queue B a minimum of 2048 M
-    out.println("<queue name=\"queueB\">");
-    out.println("<minResources>2048mb,0vcores</minResources>");
-    out.println("<aclAdministerApps>alice,bob admins</aclAdministerApps>");
-    out.println("<schedulingPolicy>fair</schedulingPolicy>");
-    out.println("</queue>");
-    // Give queue C no minimum
-    out.println("<queue name=\"queueC\">");
-    out.println("<aclSubmitApps>alice,bob admins</aclSubmitApps>");
-    out.println("</queue>");
-    // Give queue D a limit of 3 running apps
-    out.println("<queue name=\"queueD\">");
-    out.println("<maxRunningApps>3</maxRunningApps>");
-    out.println("</queue>");
-    // Give queue E a preemption timeout of one minute
-    out.println("<queue name=\"queueE\">");
-    out.println("<minSharePreemptionTimeout>60</minSharePreemptionTimeout>");
-    out.println("</queue>");
-    // Set default limit of apps per queue to 15
-    out.println("<queueMaxAppsDefault>15</queueMaxAppsDefault>");
-    // Set default limit of apps per user to 5
-    out.println("<userMaxAppsDefault>5</userMaxAppsDefault>");
-    // Give user1 a limit of 10 jobs
-    out.println("<user name=\"user1\">");
-    out.println("<maxRunningApps>10</maxRunningApps>");
-    out.println("</user>");
-    // Set default min share preemption timeout to 2 minutes
-    out.println("<defaultMinSharePreemptionTimeout>120"
-        + "</defaultMinSharePreemptionTimeout>");
-    // Set fair share preemption timeout to 5 minutes
-    out.println("<fairSharePreemptionTimeout>300</fairSharePreemptionTimeout>");
-    // Set default scheduling policy to DRF
-    out.println("<defaultQueueSchedulingPolicy>drf</defaultQueueSchedulingPolicy>");
-    out.println("</allocations>");
-    out.close();
-
-    QueueManager queueManager = scheduler.getQueueManager();
-    queueManager.initialize();
-
-    assertEquals(6, queueManager.getLeafQueues().size()); // 5 in file + default queue
-    assertEquals(Resources.createResource(0),
-        queueManager.getMinResources("root." + YarnConfiguration.DEFAULT_QUEUE_NAME));
-    assertEquals(Resources.createResource(0),
-        queueManager.getMinResources("root." + YarnConfiguration.DEFAULT_QUEUE_NAME));
-
-    assertEquals(Resources.createResource(1024, 0),
-        queueManager.getMinResources("root.queueA"));
-    assertEquals(Resources.createResource(2048, 0),
-        queueManager.getMinResources("root.queueB"));
-    assertEquals(Resources.createResource(0),
-        queueManager.getMinResources("root.queueC"));
-    assertEquals(Resources.createResource(0),
-        queueManager.getMinResources("root.queueD"));
-    assertEquals(Resources.createResource(0),
-        queueManager.getMinResources("root.queueE"));
-
-    assertEquals(15, queueManager.getQueueMaxApps("root." + YarnConfiguration.DEFAULT_QUEUE_NAME));
-    assertEquals(15, queueManager.getQueueMaxApps("root.queueA"));
-    assertEquals(15, queueManager.getQueueMaxApps("root.queueB"));
-    assertEquals(15, queueManager.getQueueMaxApps("root.queueC"));
-    assertEquals(3, queueManager.getQueueMaxApps("root.queueD"));
-    assertEquals(15, queueManager.getQueueMaxApps("root.queueE"));
-    assertEquals(10, queueManager.getUserMaxApps("user1"));
-    assertEquals(5, queueManager.getUserMaxApps("user2"));
-
-    // Root should get * ACL
-    assertEquals("*",queueManager.getQueueAcl("root",
-        QueueACL.ADMINISTER_QUEUE).getAclString());
-    assertEquals("*", queueManager.getQueueAcl("root",
-        QueueACL.SUBMIT_APPLICATIONS).getAclString());
-
-    // Unspecified queues should get default ACL
-    assertEquals(" ",queueManager.getQueueAcl("root.queueA",
-        QueueACL.ADMINISTER_QUEUE).getAclString());
-    assertEquals(" ", queueManager.getQueueAcl("root.queueA",
-        QueueACL.SUBMIT_APPLICATIONS).getAclString());
-
-    // Queue B ACL
-    assertEquals("alice,bob admins",queueManager.getQueueAcl("root.queueB",
-        QueueACL.ADMINISTER_QUEUE).getAclString());
-
-    // Queue C ACL
-    assertEquals("alice,bob admins",queueManager.getQueueAcl("root.queueC",
-        QueueACL.SUBMIT_APPLICATIONS).getAclString());
-
-    assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root." + 
-        YarnConfiguration.DEFAULT_QUEUE_NAME));
-    assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root.queueA"));
-    assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root.queueB"));
-    assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root.queueC"));
-    assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root.queueD"));
-    assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root.queueA"));
-    assertEquals(60000, queueManager.getMinSharePreemptionTimeout("root.queueE"));
-    assertEquals(300000, queueManager.getFairSharePreemptionTimeout());
-    
-    // Verify existing queues have default scheduling policy
-    assertEquals(DominantResourceFairnessPolicy.NAME,
-        queueManager.getQueue("root").getPolicy().getName());
-    assertEquals(DominantResourceFairnessPolicy.NAME,
-        queueManager.getQueue("root.queueA").getPolicy().getName());
-    // Verify default is overriden if specified explicitly
-    assertEquals(FairSharePolicy.NAME,
-        queueManager.getQueue("root.queueB").getPolicy().getName());
-    // Verify new queue gets default scheduling policy
-    assertEquals(DominantResourceFairnessPolicy.NAME,
-        queueManager.getLeafQueue("root.newqueue", true).getPolicy().getName());
-  }
-
   @Test
   public void testHierarchicalQueueAllocationFileParsing() throws IOException, SAXException, 
       AllocationConfigurationException, ParserConfigurationException {
-    Configuration conf = createConfiguration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
@@ -980,9 +877,9 @@ public class TestFairScheduler {
     out.println("</allocations>");
     out.close();
 
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     QueueManager queueManager = scheduler.getQueueManager();
-    queueManager.initialize();
-    
     Collection<FSLeafQueue> leafQueues = queueManager.getLeafQueues();
     Assert.assertEquals(4, leafQueues.size());
     Assert.assertNotNull(queueManager.getLeafQueue("queueA", false));
@@ -995,9 +892,7 @@ public class TestFairScheduler {
   
   @Test
   public void testConfigureRootQueue() throws Exception {
-    Configuration conf = createConfiguration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
@@ -1014,9 +909,9 @@ public class TestFairScheduler {
     out.println("</queue>");
     out.println("</allocations>");
     out.close();
-
+    
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
     QueueManager queueManager = scheduler.getQueueManager();
-    queueManager.initialize();
     
     FSQueue root = queueManager.getRootQueue();
     assertTrue(root.getPolicy() instanceof DominantResourceFairnessPolicy);
@@ -1025,136 +920,9 @@ public class TestFairScheduler {
     assertNotNull(queueManager.getLeafQueue("child2", false));
   }
   
-  /**
-   * Verify that you can't place queues at the same level as the root queue in
-   * the allocations file.
-   */
-  @Test (expected = AllocationConfigurationException.class)
-  public void testQueueAlongsideRoot() throws Exception {
-    Configuration conf = createConfiguration();
-    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
-
-    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
-    out.println("<?xml version=\"1.0\"?>");
-    out.println("<allocations>");
-    out.println("<queue name=\"root\">");
-    out.println("</queue>");
-    out.println("<queue name=\"other\">");
-    out.println("</queue>");
-    out.println("</allocations>");
-    out.close();
-
-    QueueManager queueManager = scheduler.getQueueManager();
-    queueManager.initialize();
-  }
-  
-  @Test
-  public void testBackwardsCompatibleAllocationFileParsing() throws Exception {
-    Configuration conf = createConfiguration();
-    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
-
-    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
-    out.println("<?xml version=\"1.0\"?>");
-    out.println("<allocations>");
-    // Give queue A a minimum of 1024 M
-    out.println("<pool name=\"queueA\">");
-    out.println("<minResources>1024mb,0vcores</minResources>");
-    out.println("</pool>");
-    // Give queue B a minimum of 2048 M
-    out.println("<pool name=\"queueB\">");
-    out.println("<minResources>2048mb,0vcores</minResources>");
-    out.println("<aclAdministerApps>alice,bob admins</aclAdministerApps>");
-    out.println("</pool>");
-    // Give queue C no minimum
-    out.println("<pool name=\"queueC\">");
-    out.println("<aclSubmitApps>alice,bob admins</aclSubmitApps>");
-    out.println("</pool>");
-    // Give queue D a limit of 3 running apps
-    out.println("<pool name=\"queueD\">");
-    out.println("<maxRunningApps>3</maxRunningApps>");
-    out.println("</pool>");
-    // Give queue E a preemption timeout of one minute
-    out.println("<pool name=\"queueE\">");
-    out.println("<minSharePreemptionTimeout>60</minSharePreemptionTimeout>");
-    out.println("</pool>");
-    // Set default limit of apps per queue to 15
-    out.println("<queueMaxAppsDefault>15</queueMaxAppsDefault>");
-    // Set default limit of apps per user to 5
-    out.println("<userMaxAppsDefault>5</userMaxAppsDefault>");
-    // Give user1 a limit of 10 jobs
-    out.println("<user name=\"user1\">");
-    out.println("<maxRunningApps>10</maxRunningApps>");
-    out.println("</user>");
-    // Set default min share preemption timeout to 2 minutes
-    out.println("<defaultMinSharePreemptionTimeout>120"
-        + "</defaultMinSharePreemptionTimeout>");
-    // Set fair share preemption timeout to 5 minutes
-    out.println("<fairSharePreemptionTimeout>300</fairSharePreemptionTimeout>");
-    out.println("</allocations>");
-    out.close();
-
-    QueueManager queueManager = scheduler.getQueueManager();
-    queueManager.initialize();
-
-    assertEquals(6, queueManager.getLeafQueues().size()); // 5 in file + default queue
-    assertEquals(Resources.createResource(0),
-        queueManager.getMinResources("root." + YarnConfiguration.DEFAULT_QUEUE_NAME));
-    assertEquals(Resources.createResource(0),
-        queueManager.getMinResources("root." + YarnConfiguration.DEFAULT_QUEUE_NAME));
-
-    assertEquals(Resources.createResource(1024, 0),
-        queueManager.getMinResources("root.queueA"));
-    assertEquals(Resources.createResource(2048, 0),
-        queueManager.getMinResources("root.queueB"));
-    assertEquals(Resources.createResource(0),
-        queueManager.getMinResources("root.queueC"));
-    assertEquals(Resources.createResource(0),
-        queueManager.getMinResources("root.queueD"));
-    assertEquals(Resources.createResource(0),
-        queueManager.getMinResources("root.queueE"));
-
-    assertEquals(15, queueManager.getQueueMaxApps("root." + YarnConfiguration.DEFAULT_QUEUE_NAME));
-    assertEquals(15, queueManager.getQueueMaxApps("root.queueA"));
-    assertEquals(15, queueManager.getQueueMaxApps("root.queueB"));
-    assertEquals(15, queueManager.getQueueMaxApps("root.queueC"));
-    assertEquals(3, queueManager.getQueueMaxApps("root.queueD"));
-    assertEquals(15, queueManager.getQueueMaxApps("root.queueE"));
-    assertEquals(10, queueManager.getUserMaxApps("user1"));
-    assertEquals(5, queueManager.getUserMaxApps("user2"));
-
-    // Unspecified queues should get default ACL
-    assertEquals(" ", queueManager.getQueueAcl("root.queueA",
-        QueueACL.ADMINISTER_QUEUE).getAclString());
-    assertEquals(" ", queueManager.getQueueAcl("root.queueA",
-        QueueACL.SUBMIT_APPLICATIONS).getAclString());
-
-    // Queue B ACL
-    assertEquals("alice,bob admins", queueManager.getQueueAcl("root.queueB",
-        QueueACL.ADMINISTER_QUEUE).getAclString());
-
-    // Queue C ACL
-    assertEquals("alice,bob admins", queueManager.getQueueAcl("root.queueC",
-        QueueACL.SUBMIT_APPLICATIONS).getAclString());
-
-
-    assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root." +
-        YarnConfiguration.DEFAULT_QUEUE_NAME));
-    assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root.queueA"));
-    assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root.queueB"));
-    assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root.queueC"));
-    assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root.queueD"));
-    assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root.queueA"));
-    assertEquals(60000, queueManager.getMinSharePreemptionTimeout("root.queueE"));
-    assertEquals(300000, queueManager.getFairSharePreemptionTimeout());
-  }
-
   @Test (timeout = 5000)
   public void testIsStarvedForMinShare() throws Exception {
-    Configuration conf = createConfiguration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
@@ -1168,8 +936,7 @@ public class TestFairScheduler {
     out.println("</allocations>");
     out.close();
 
-    QueueManager queueManager = scheduler.getQueueManager();
-    queueManager.initialize();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     // Add one big node (only care about aggregate capacity)
     RMNode node1 =
@@ -1212,9 +979,7 @@ public class TestFairScheduler {
 
   @Test (timeout = 5000)
   public void testIsStarvedForFairShare() throws Exception {
-    Configuration conf = createConfiguration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
@@ -1228,9 +993,8 @@ public class TestFairScheduler {
     out.println("</allocations>");
     out.close();
 
-    QueueManager queueManager = scheduler.getQueueManager();
-    queueManager.initialize();
-
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+    
     // Add one big node (only care about aggregate capacity)
     RMNode node1 =
         MockNodes.newNodeInfo(1, Resources.createResource(4 * 1024, 4), 1,
@@ -1277,13 +1041,9 @@ public class TestFairScheduler {
    * now this means decreasing order of priority.
    */
   public void testChoiceOfPreemptedContainers() throws Exception {
-    Configuration conf = createConfiguration();
-    
     conf.setLong(FairSchedulerConfiguration.PREEMPTION_INTERVAL, 5000);
     conf.setLong(FairSchedulerConfiguration.WAIT_TIME_BEFORE_KILL, 10000); 
-    
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE + ".allocation.file", ALLOC_FILE);
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
     
     MockClock clock = new MockClock();
     scheduler.setClock(clock);
@@ -1305,9 +1065,8 @@ public class TestFairScheduler {
     out.println("</queue>");
     out.println("</allocations>");
     out.close();
-
-    QueueManager queueManager = scheduler.getQueueManager();
-    queueManager.initialize();
+    
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     // Create four nodes
     RMNode node1 =
@@ -1443,15 +1202,16 @@ public class TestFairScheduler {
    * Tests the timing of decision to preempt tasks.
    */
   public void testPreemptionDecision() throws Exception {
-    Configuration conf = createConfiguration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
     MockClock clock = new MockClock();
     scheduler.setClock(clock);
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
     out.println("<allocations>");
+    out.println("<queue name=\"default\">");
+    out.println("<maxResources>0mb,0vcores</maxResources>");
+    out.println("</queue>");
     out.println("<queue name=\"queueA\">");
     out.println("<weight>.25</weight>");
     out.println("<minResources>1024mb,0vcores</minResources>");
@@ -1473,8 +1233,7 @@ public class TestFairScheduler {
     out.println("</allocations>");
     out.close();
 
-    QueueManager queueManager = scheduler.getQueueManager();
-    queueManager.initialize();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     // Create four nodes
     RMNode node1 =
@@ -1570,7 +1329,9 @@ public class TestFairScheduler {
   }
   
   @Test (timeout = 5000)
-  public void testMultipleContainersWaitingForReservation() {
+  public void testMultipleContainersWaitingForReservation() throws IOException {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     // Add a node
     RMNode node1 =
         MockNodes
@@ -1600,9 +1361,7 @@ public class TestFairScheduler {
   @Test (timeout = 5000)
   public void testUserMaxRunningApps() throws Exception {
     // Set max running apps
-    Configuration conf = createConfiguration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
@@ -1613,8 +1372,7 @@ public class TestFairScheduler {
     out.println("</allocations>");
     out.close();
 
-    QueueManager queueManager = scheduler.getQueueManager();
-    queueManager.initialize();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
     
     // Add a node
     RMNode node1 =
@@ -1654,7 +1412,9 @@ public class TestFairScheduler {
   }
   
   @Test (timeout = 5000)
-  public void testReservationWhileMultiplePriorities() {
+  public void testReservationWhileMultiplePriorities() throws IOException {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     // Add a node
     RMNode node1 =
         MockNodes
@@ -1717,9 +1477,7 @@ public class TestFairScheduler {
   @Test
   public void testAclSubmitApplication() throws Exception {
     // Set acl's
-    Configuration conf = createConfiguration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
@@ -1735,8 +1493,7 @@ public class TestFairScheduler {
     out.println("</allocations>");
     out.close();
 
-    QueueManager queueManager = scheduler.getQueueManager();
-    queueManager.initialize();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
     
     ApplicationAttemptId attId1 = createSchedulingRequest(1024, "queue1",
         "norealuserhasthisname", 1);
@@ -1751,6 +1508,8 @@ public class TestFairScheduler {
   
   @Test (timeout = 5000)
   public void testMultipleNodesSingleRackRequest() throws Exception {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     RMNode node1 =
         MockNodes
             .newNodeInfo(1, Resources.createResource(1024), 1, "127.0.0.1");
@@ -1797,6 +1556,8 @@ public class TestFairScheduler {
   
   @Test (timeout = 5000)
   public void testFifoWithinQueue() throws Exception {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     RMNode node1 =
         MockNodes
             .newNodeInfo(1, Resources.createResource(3072, 3), 1, "127.0.0.1");
@@ -1837,11 +1598,9 @@ public class TestFairScheduler {
   }
 
   @Test(timeout = 3000)
-  public void testMaxAssign() throws AllocationConfigurationException {
-    // set required scheduler configs
-    scheduler.assignMultiple = true;
-    scheduler.getQueueManager().getLeafQueue("root.default", true)
-        .setPolicy(SchedulingPolicy.getDefault());
+  public void testMaxAssign() throws Exception {
+    conf.setBoolean(FairSchedulerConfiguration.ASSIGN_MULTIPLE, true);
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     RMNode node =
         MockNodes.newNodeInfo(1, Resources.createResource(16384, 16), 0,
@@ -1884,6 +1643,8 @@ public class TestFairScheduler {
    */
   @Test(timeout = 5000)
   public void testAssignContainer() throws Exception {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     final String user = "user1";
     final String fifoQueue = "fifo";
     final String fairParent = "fairParent";
@@ -1951,9 +1712,7 @@ public class TestFairScheduler {
   @Test
   public void testNotAllowSubmitApplication() throws Exception {
     // Set acl's
-    Configuration conf = createConfiguration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
     out.println("<allocations>");
@@ -1967,8 +1726,8 @@ public class TestFairScheduler {
     out.println("</queue>");
     out.println("</allocations>");
     out.close();
-    QueueManager queueManager = scheduler.getQueueManager();
-    queueManager.initialize();
+    
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
     
     int appId = this.APP_ID++;
     String user = "usernotallow";
@@ -2017,7 +1776,9 @@ public class TestFairScheduler {
   }
   
   @Test
-  public void testReservationThatDoesntFit() {
+  public void testReservationThatDoesntFit() throws IOException {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     RMNode node1 =
         MockNodes
             .newNodeInfo(1, Resources.createResource(1024), 1, "127.0.0.1");
@@ -2043,7 +1804,9 @@ public class TestFairScheduler {
   }
   
   @Test
-  public void testRemoveNodeUpdatesRootQueueMetrics() {
+  public void testRemoveNodeUpdatesRootQueueMetrics() throws IOException {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     assertEquals(0, scheduler.getRootQueueMetrics().getAvailableMB());
 	assertEquals(0, scheduler.getRootQueueMetrics().getAvailableVirtualCores());
     
@@ -2069,7 +1832,9 @@ public class TestFairScheduler {
 }
 
   @Test
-  public void testStrictLocality() {
+  public void testStrictLocality() throws IOException {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024), 1, "127.0.0.1");
     NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
     scheduler.handle(nodeEvent1);
@@ -2107,7 +1872,9 @@ public class TestFairScheduler {
   }
   
   @Test
-  public void testCancelStrictLocality() {
+  public void testCancelStrictLocality() throws IOException {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024), 1, "127.0.0.1");
     NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
     scheduler.handle(nodeEvent1);
@@ -2155,7 +1922,9 @@ public class TestFairScheduler {
    * a reservation on another.
    */
   @Test
-  public void testReservationsStrictLocality() {
+  public void testReservationsStrictLocality() throws IOException {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024), 1, "127.0.0.1");
     RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(1024), 2, "127.0.0.2");
     NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node1);
@@ -2193,7 +1962,9 @@ public class TestFairScheduler {
   }
   
   @Test
-  public void testNoMoreCpuOnNode() {
+  public void testNoMoreCpuOnNode() throws IOException {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(2048, 1),
         1, "127.0.0.1");
     NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
@@ -2213,6 +1984,8 @@ public class TestFairScheduler {
 
   @Test
   public void testBasicDRFAssignment() throws Exception {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     RMNode node = MockNodes.newNodeInfo(1, BuilderUtils.newResource(8192, 5));
     NodeAddedSchedulerEvent nodeEvent = new NodeAddedSchedulerEvent(node);
     scheduler.handle(nodeEvent);
@@ -2251,6 +2024,8 @@ public class TestFairScheduler {
    */
   @Test
   public void testBasicDRFWithQueues() throws Exception {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     RMNode node = MockNodes.newNodeInfo(1, BuilderUtils.newResource(8192, 7),
         1, "127.0.0.1");
     NodeAddedSchedulerEvent nodeEvent = new NodeAddedSchedulerEvent(node);
@@ -2285,6 +2060,8 @@ public class TestFairScheduler {
   
   @Test
   public void testDRFHierarchicalQueues() throws Exception {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     RMNode node = MockNodes.newNodeInfo(1, BuilderUtils.newResource(12288, 12),
         1, "127.0.0.1");
     NodeAddedSchedulerEvent nodeEvent = new NodeAddedSchedulerEvent(node);
@@ -2349,9 +2126,9 @@ public class TestFairScheduler {
 
   @Test(timeout = 30000)
   public void testHostPortNodeName() throws Exception {
-    scheduler.getConf().setBoolean(YarnConfiguration
+    conf.setBoolean(YarnConfiguration
         .RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME, true);
-    scheduler.reinitialize(scheduler.getConf(), 
+    scheduler.reinitialize(conf, 
         resourceManager.getRMContext());
     RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024),
         1, "127.0.0.1", 1);
@@ -2426,9 +2203,7 @@ public class TestFairScheduler {
   
   @Test
   public void testUserAndQueueMaxRunningApps() throws Exception {
-    Configuration conf = createConfiguration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
@@ -2442,8 +2217,7 @@ public class TestFairScheduler {
     out.println("</allocations>");
     out.close();
     
-    QueueManager queueManager = scheduler.getQueueManager();
-    queueManager.initialize();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
     
     // exceeds no limits
     ApplicationAttemptId attId1 = createSchedulingRequest(1024, "queue1", "user1");
@@ -2479,9 +2253,7 @@ public class TestFairScheduler {
   
   @Test
   public void testMaxRunningAppsHierarchicalQueues() throws Exception {
-    Configuration conf = createConfiguration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
     MockClock clock = new MockClock();
     scheduler.setClock(clock);
 
@@ -2499,8 +2271,7 @@ public class TestFairScheduler {
     out.println("</allocations>");
     out.close();
     
-    QueueManager queueManager = scheduler.getQueueManager();
-    queueManager.initialize();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
     
     // exceeds no limits
     ApplicationAttemptId attId1 = createSchedulingRequest(1024, "queue1.sub1", "user1");
@@ -2629,10 +2400,8 @@ public class TestFairScheduler {
   
   @Test
   public void testDontAllowUndeclaredPools() throws Exception{
-    Configuration conf = createConfiguration();
     conf.setBoolean(FairSchedulerConfiguration.ALLOW_UNDECLARED_POOLS, false);
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
-    scheduler.reinitialize(conf, resourceManager.getRMContext());
 
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
@@ -2642,8 +2411,8 @@ public class TestFairScheduler {
     out.println("</allocations>");
     out.close();
 
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
     QueueManager queueManager = scheduler.getQueueManager();
-    queueManager.initialize();
     
     FSLeafQueue jerryQueue = queueManager.getLeafQueue("jerry", false);
     FSLeafQueue defaultQueue = queueManager.getLeafQueue("default", false);
@@ -2672,6 +2441,8 @@ public class TestFairScheduler {
   @SuppressWarnings("resource")
   @Test
   public void testBlacklistNodes() throws Exception {
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
     final int GB = 1024;
     String host = "127.0.0.1";
     RMNode node =

+ 0 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerConfiguration.java

@@ -61,14 +61,4 @@ public class TestFairSchedulerConfiguration {
     parseResourceConfigValue("1o24vc0res");
   }
   
-  @Test
-  public void testGetAllocationFileFromClasspath() {
-    FairSchedulerConfiguration conf = new FairSchedulerConfiguration(
-        new Configuration());
-    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE,
-        "test-fair-scheduler.xml");
-    File allocationFile = conf.getAllocationFile();
-    Assert.assertEquals("test-fair-scheduler.xml", allocationFile.getName());
-    Assert.assertTrue(allocationFile.exists());
-  }
 }

+ 9 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestMaxRunningAppsEnforcer.java

@@ -39,18 +39,21 @@ public class TestMaxRunningAppsEnforcer {
   
   @Before
   public void setup() throws Exception {
+    Configuration conf = new Configuration();
     clock = new TestFairScheduler.MockClock();
     FairScheduler scheduler = mock(FairScheduler.class);
     when(scheduler.getConf()).thenReturn(
-        new FairSchedulerConfiguration(new Configuration()));
+        new FairSchedulerConfiguration(conf));
     when(scheduler.getClock()).thenReturn(clock);
+    AllocationConfiguration allocConf = new AllocationConfiguration(
+        conf);
+    when(scheduler.getAllocationConfiguration()).thenReturn(allocConf);
     
     queueManager = new QueueManager(scheduler);
-    queueManager.initialize();
-    
-    queueMaxApps = queueManager.info.queueMaxApps;
-    userMaxApps = queueManager.info.userMaxApps;
-    maxAppsEnforcer = new MaxRunningAppsEnforcer(queueManager);
+    queueManager.initialize(conf);
+    queueMaxApps = allocConf.queueMaxApps;
+    userMaxApps = allocConf.userMaxApps;
+    maxAppsEnforcer = new MaxRunningAppsEnforcer(scheduler);
     appNum = 0;
   }