浏览代码

Merge trunk to HDFS-4685.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-4685@1549699 13f79535-47bb-0310-9956-ffa450edef68
Chris Nauroth 11 年之前
父节点
当前提交
a6313b1523
共有 100 个文件被更改,包括 2148 次插入756 次删除
  1. 3 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 8 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
  3. 44 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java
  4. 15 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
  5. 14 0
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  6. 49 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestGroupsCaching.java
  7. 15 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  8. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  9. 119 20
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  10. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/BasicInetPeer.java
  11. 15 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/DomainPeer.java
  12. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/EncryptedPeer.java
  13. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/NioInetPeer.java
  14. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/Peer.java
  15. 53 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
  16. 7 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
  17. 25 42
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeHttpServer.java
  18. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
  19. 1 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  20. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  21. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
  22. 12 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
  23. 20 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java
  24. 2 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  25. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  26. 10 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java
  27. 0 18
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  28. 121 44
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  29. 10 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  30. 14 50
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
  31. 51 44
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  32. 22 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
  33. 8 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
  34. 14 24
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
  35. 11 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
  36. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
  37. 71 84
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
  38. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectorySnapshottable.java
  39. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java
  40. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
  41. 6 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
  42. 8 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
  43. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  44. 20 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserParam.java
  45. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
  46. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  47. 13 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
  48. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelShortCircuitReadUnCached.java
  49. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPeerCache.java
  50. 52 34
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniJournalCluster.java
  51. 1 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
  52. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
  53. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
  54. 9 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java
  55. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java
  56. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java
  57. 10 15
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
  58. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
  59. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
  60. 28 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
  61. 15 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java
  62. 5 0
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
  63. 15 0
      hadoop-yarn-project/CHANGES.txt
  64. 31 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
  65. 49 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
  66. 10 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  67. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
  68. 0 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/pom.xml
  69. 25 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/test/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/TestUnmanagedAMLauncher.java
  70. 86 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
  71. 166 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java
  72. 15 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  73. 73 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateRequest.java
  74. 114 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java
  75. 53 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
  76. 35 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
  77. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
  78. 9 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
  79. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java
  80. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/NullRMStateStore.java
  81. 4 25
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
  82. 14 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
  83. 22 20
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  84. 0 36
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppRemovedEvent.java
  85. 3 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
  86. 10 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
  87. 9 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
  88. 11 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
  89. 10 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
  90. 10 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
  91. 13 0
      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
  92. 10 0
      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
  93. 9 1
      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
  94. 12 0
      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
  95. 15 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
  96. 91 20
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
  97. 27 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
  98. 123 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
  99. 44 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
  100. 20 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java

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

@@ -487,6 +487,9 @@ Release 2.3.0 - UNRELEASED
 
   OPTIMIZATIONS
 
+    HADOOP-10142. Avoid groups lookup for unprivileged users such as "dr.who"
+    (vinay via cmccabe)
+
   BUG FIXES
 
     HADOOP-10028. Malformed ssl-server.xml.example. (Haohui Mai via jing9)

+ 8 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java

@@ -204,6 +204,14 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   public static final String DEFAULT_HADOOP_HTTP_STATIC_USER =
     "dr.who";
 
+  /**
+   * User->groups static mapping to override the groups lookup
+   */
+  public static final String HADOOP_USER_GROUP_STATIC_OVERRIDES = 
+      "hadoop.user.group.static.mapping.overrides";
+  public static final String HADOOP_USER_GROUP_STATIC_OVERRIDES_DEFAULT =
+      "dr.who=;";
+
   /** Enable/Disable aliases serving from jetty */
   public static final String HADOOP_JETTY_LOGS_SERVE_ALIASES =
     "hadoop.jetty.logs.serve.aliases";

+ 44 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java

@@ -18,15 +18,20 @@
 package org.apache.hadoop.security;
 
 import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 
 import org.apache.commons.logging.Log;
@@ -49,6 +54,8 @@ public class Groups {
   
   private final Map<String, CachedGroups> userToGroupsMap = 
     new ConcurrentHashMap<String, CachedGroups>();
+  private final Map<String, List<String>> staticUserToGroupsMap = 
+      new HashMap<String, List<String>>();
   private final long cacheTimeout;
   private final long warningDeltaMs;
 
@@ -66,12 +73,43 @@ public class Groups {
     warningDeltaMs =
       conf.getLong(CommonConfigurationKeys.HADOOP_SECURITY_GROUPS_CACHE_WARN_AFTER_MS,
         CommonConfigurationKeys.HADOOP_SECURITY_GROUPS_CACHE_WARN_AFTER_MS_DEFAULT);
-    
+    parseStaticMapping(conf);
+
     if(LOG.isDebugEnabled())
       LOG.debug("Group mapping impl=" + impl.getClass().getName() + 
           "; cacheTimeout=" + cacheTimeout + "; warningDeltaMs=" +
           warningDeltaMs);
   }
+
+  /*
+   * Parse the hadoop.user.group.static.mapping.overrides configuration to
+   * staticUserToGroupsMap
+   */
+  private void parseStaticMapping(Configuration conf) {
+    String staticMapping = conf.get(
+        CommonConfigurationKeys.HADOOP_USER_GROUP_STATIC_OVERRIDES,
+        CommonConfigurationKeys.HADOOP_USER_GROUP_STATIC_OVERRIDES_DEFAULT);
+    Collection<String> mappings = StringUtils.getStringCollection(
+        staticMapping, ";");
+    for (String users : mappings) {
+      Collection<String> userToGroups = StringUtils.getStringCollection(users,
+          "=");
+      if (userToGroups.size() < 1 || userToGroups.size() > 2) {
+        throw new HadoopIllegalArgumentException("Configuration "
+            + CommonConfigurationKeys.HADOOP_USER_GROUP_STATIC_OVERRIDES
+            + " is invalid");
+      }
+      String[] userToGroupsArray = userToGroups.toArray(new String[userToGroups
+          .size()]);
+      String user = userToGroupsArray[0];
+      List<String> groups = Collections.emptyList();
+      if (userToGroupsArray.length == 2) {
+        groups = (List<String>) StringUtils
+            .getStringCollection(userToGroupsArray[1]);
+      }
+      staticUserToGroupsMap.put(user, groups);
+    }
+  }
   
   /**
    * Get the group memberships of a given user.
@@ -80,6 +118,11 @@ public class Groups {
    * @throws IOException
    */
   public List<String> getGroups(String user) throws IOException {
+    // No need to lookup for groups of static users
+    List<String> staticMapping = staticUserToGroupsMap.get(user);
+    if (staticMapping != null) {
+      return staticMapping;
+    }
     // Return cached value if available
     CachedGroups groups = userToGroupsMap.get(user);
     long startMs = Time.monotonicNow();

+ 15 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java

@@ -325,10 +325,24 @@ public class StringUtils {
    * @return an <code>ArrayList</code> of string values
    */
   public static Collection<String> getStringCollection(String str){
+    String delim = ",";
+    return getStringCollection(str, delim);
+  }
+
+  /**
+   * Returns a collection of strings.
+   * 
+   * @param str
+   *          String to parse
+   * @param delim
+   *          delimiter to separate the values
+   * @return Collection of parsed elements.
+   */
+  public static Collection<String> getStringCollection(String str, String delim) {
     List<String> values = new ArrayList<String>();
     if (str == null)
       return values;
-    StringTokenizer tokenizer = new StringTokenizer (str,",");
+    StringTokenizer tokenizer = new StringTokenizer(str, delim);
     values = new ArrayList<String>();
     while (tokenizer.hasMoreTokens()) {
       values.add(tokenizer.nextToken());

+ 14 - 0
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -1261,4 +1261,18 @@
       Specify the port number used by Hadoop mount daemon.
   </description>
 </property>
+
+<property>
+  <name>hadoop.user.group.static.mapping.overrides</name>
+  <value>dr.who=;</value>
+  <description>
+    Static mapping of user to groups. This will override the groups if
+    available in the system for the specified user. In otherwords, groups
+    look-up will not happen for these users, instead groups mapped in this
+    configuration will be used.
+    Mapping should be in this format.
+    user1=group1,group2;user2=;user3=group2;
+    Default, "dr.who=;" will consider "dr.who" as user without groups.
+  </description>
+</property>
 </configuration>

+ 49 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestGroupsCaching.java

@@ -19,14 +19,17 @@ package org.apache.hadoop.security;
 
 import java.io.IOException;
 
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
 
+import org.junit.Before;
 import org.junit.Test;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
 
 import org.apache.commons.logging.Log;
@@ -40,10 +43,12 @@ import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
 
 public class TestGroupsCaching {
   public static final Log LOG = LogFactory.getLog(TestGroupsCaching.class);
-  private static Configuration conf = new Configuration();
   private static String[] myGroups = {"grp1", "grp2"};
+  private Configuration conf;
 
-  static {
+  @Before
+  public void setup() {
+    conf = new Configuration();
     conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
       FakeGroupMapping.class,
       ShellBasedUnixGroupsMapping.class);
@@ -88,7 +93,7 @@ public class TestGroupsCaching {
   }
 
   @Test
-  public void TestGroupsCaching() throws Exception {
+  public void testGroupsCaching() throws Exception {
     Groups groups = new Groups(conf);
     groups.cacheGroupsAdd(Arrays.asList(myGroups));
     groups.refresh();
@@ -117,4 +122,45 @@ public class TestGroupsCaching {
     FakeGroupMapping.clearBlackList();
     assertTrue(groups.getGroups("user1").size() == 2);
   }
+
+  public static class FakeunPrivilegedGroupMapping extends FakeGroupMapping {
+    private static boolean invoked = false;
+    @Override
+    public List<String> getGroups(String user) throws IOException {
+      invoked = true;
+      return super.getGroups(user);
+    }
+  }
+
+  /*
+   * Group lookup should not happen for static users
+   */
+  @Test
+  public void testGroupLookupForStaticUsers() throws Exception {
+    conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
+        FakeunPrivilegedGroupMapping.class, ShellBasedUnixGroupsMapping.class);
+    conf.set(CommonConfigurationKeys.HADOOP_USER_GROUP_STATIC_OVERRIDES, "me=;user1=group1;user2=group1,group2");
+    Groups groups = new Groups(conf);
+    List<String> userGroups = groups.getGroups("me");
+    assertTrue("non-empty groups for static user", userGroups.isEmpty());
+    assertFalse("group lookup done for static user",
+        FakeunPrivilegedGroupMapping.invoked);
+    
+    List<String> expected = new ArrayList<String>();
+    expected.add("group1");
+
+    FakeunPrivilegedGroupMapping.invoked = false;
+    userGroups = groups.getGroups("user1");
+    assertTrue("groups not correct", expected.equals(userGroups));
+    assertFalse("group lookup done for unprivileged user",
+        FakeunPrivilegedGroupMapping.invoked);
+
+    expected.add("group2");
+    FakeunPrivilegedGroupMapping.invoked = false;
+    userGroups = groups.getGroups("user2");
+    assertTrue("groups not correct", expected.equals(userGroups));
+    assertFalse("group lookup done for unprivileged user",
+        FakeunPrivilegedGroupMapping.invoked);
+
+  }
 }

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

@@ -233,6 +233,15 @@ Trunk (Unreleased)
 
     HDFS-5630. Hook up cache directive and pool usage statistics. (wang)
 
+    HDFS-5312. Generate HTTP / HTTPS URL in DFSUtil#getInfoServer() based on the 
+    configured http policy. (Haohui Mai via jing9)
+
+    HDFS-5554. Flatten INodeFile hierarchy: Replace INodeFileWithSnapshot with
+    FileWithSnapshotFeature.  (jing9 via szetszwo)
+
+    HDFS-5629. Support HTTPS in JournalNode and SecondaryNameNode. 
+    (Haohui Mai via jing9)
+
   OPTIMIZATIONS
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
 
@@ -578,6 +587,9 @@ Release 2.4.0 - UNRELEASED
 
     HDFS-5633. Improve OfflineImageViewer to use less memory. (jing9)
 
+    HDFS-4983. Numeric usernames do not work with WebHDFS FS. (Yongjun Zhang via
+    jing9)
+
   OPTIMIZATIONS
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
@@ -789,6 +801,9 @@ Release 2.3.0 - UNRELEASED
     HDFS-5590. Block ID and generation stamp may be reused when persistBlocks is 
     set to false. (jing9)
 
+    HDFS-5353. Short circuit reads fail when dfs.encrypt.data.transfer is 
+    enabled. (Colin Patrick McCabe via jing9)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

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

@@ -130,6 +130,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_NAMENODE_SAFEMODE_MIN_DATANODES_DEFAULT = 0;
   public static final String  DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY = "dfs.namenode.secondary.http-address";
   public static final String  DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_DEFAULT = "0.0.0.0:50090";
+  public static final String  DFS_NAMENODE_SECONDARY_HTTPS_ADDRESS_KEY = "dfs.namenode.secondary.https-address";
+  public static final String  DFS_NAMENODE_SECONDARY_HTTPS_ADDRESS_DEFAULT = "0.0.0.0:50091";
   public static final String  DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY = "dfs.namenode.checkpoint.check.period";
   public static final long    DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_DEFAULT = 60;
   public static final String  DFS_NAMENODE_CHECKPOINT_PERIOD_KEY = "dfs.namenode.checkpoint.period";
@@ -164,6 +166,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_DEFAULT = 4;
   public static final String  DFS_WEBHDFS_ENABLED_KEY = "dfs.webhdfs.enabled";
   public static final boolean DFS_WEBHDFS_ENABLED_DEFAULT = true;
+  public static final String  DFS_WEBHDFS_USER_PATTERN_KEY = "dfs.webhdfs.user.provider.user.pattern";
+  public static final String  DFS_WEBHDFS_USER_PATTERN_DEFAULT = "^[A-Za-z_][A-Za-z0-9._-]*[$]?$";
   public static final String  DFS_PERMISSIONS_ENABLED_KEY = "dfs.permissions.enabled";
   public static final boolean DFS_PERMISSIONS_ENABLED_DEFAULT = true;
   public static final String  DFS_PERMISSIONS_SUPERUSERGROUP_KEY = "dfs.permissions.superusergroup";
@@ -502,6 +506,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_JOURNALNODE_HTTP_ADDRESS_KEY = "dfs.journalnode.http-address";
   public static final int     DFS_JOURNALNODE_HTTP_PORT_DEFAULT = 8480;
   public static final String  DFS_JOURNALNODE_HTTP_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_JOURNALNODE_HTTP_PORT_DEFAULT;
+  public static final String  DFS_JOURNALNODE_HTTPS_ADDRESS_KEY = "dfs.journalnode.https-address";
+  public static final int     DFS_JOURNALNODE_HTTPS_PORT_DEFAULT = 8481;
+  public static final String  DFS_JOURNALNODE_HTTPS_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_JOURNALNODE_HTTPS_PORT_DEFAULT;
 
   public static final String  DFS_JOURNALNODE_KEYTAB_FILE_KEY = "dfs.journalnode.keytab.file";
   public static final String  DFS_JOURNALNODE_USER_NAME_KEY = "dfs.journalnode.kerberos.principal";

+ 119 - 20
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hdfs;
 
+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 static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
@@ -89,9 +90,11 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
@@ -958,39 +961,71 @@ public class DFSUtil {
    * given namenode rpc address.
    * @param conf
    * @param namenodeAddr - namenode RPC address
-   * @param httpsAddress -If true, and if security is enabled, returns server 
-   *                      https address. If false, returns server http address.
+   * @param scheme - the scheme (http / https)
    * @return server http or https address
    * @throws IOException 
    */
-  public static String getInfoServer(InetSocketAddress namenodeAddr,
-      Configuration conf, boolean httpsAddress) throws IOException {
-    boolean securityOn = UserGroupInformation.isSecurityEnabled();
-    String httpAddressKey = (securityOn && httpsAddress) ? 
-        DFS_NAMENODE_HTTPS_ADDRESS_KEY : DFS_NAMENODE_HTTP_ADDRESS_KEY;
-    String httpAddressDefault = (securityOn && httpsAddress) ? 
-        DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT : DFS_NAMENODE_HTTP_ADDRESS_DEFAULT;
-      
-    String suffixes[];
+  public static URI getInfoServer(InetSocketAddress namenodeAddr,
+      Configuration conf, String scheme) throws IOException {
+    String[] suffixes = null;
     if (namenodeAddr != null) {
       // if non-default namenode, try reverse look up 
       // the nameServiceID if it is available
       suffixes = getSuffixIDs(conf, namenodeAddr,
           DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
           DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY);
+    }
+
+    String authority;
+    if ("http".equals(scheme)) {
+      authority = getSuffixedConf(conf, DFS_NAMENODE_HTTP_ADDRESS_KEY,
+          DFS_NAMENODE_HTTP_ADDRESS_DEFAULT, suffixes);
+    } else if ("https".equals(scheme)) {
+      authority = getSuffixedConf(conf, DFS_NAMENODE_HTTPS_ADDRESS_KEY,
+          DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT, suffixes);
     } else {
-      suffixes = new String[2];
+      throw new IllegalArgumentException("Invalid scheme:" + scheme);
     }
-    String configuredInfoAddr = getSuffixedConf(conf, httpAddressKey,
-        httpAddressDefault, suffixes);
+
     if (namenodeAddr != null) {
-      return substituteForWildcardAddress(configuredInfoAddr,
+      authority = substituteForWildcardAddress(authority,
           namenodeAddr.getHostName());
-    } else {
-      return configuredInfoAddr;
     }
+    return URI.create(scheme + "://" + authority);
+  }
+
+  /**
+   * Lookup the HTTP / HTTPS address of the namenode, and replace its hostname
+   * with defaultHost when it found out that the address is a wildcard / local
+   * address.
+   *
+   * @param defaultHost
+   *          The default host name of the namenode.
+   * @param conf
+   *          The configuration
+   * @param scheme
+   *          HTTP or HTTPS
+   * @throws IOException
+   */
+  public static URI getInfoServerWithDefaultHost(String defaultHost,
+      Configuration conf, final String scheme) throws IOException {
+    URI configuredAddr = getInfoServer(null, conf, scheme);
+    String authority = substituteForWildcardAddress(
+        configuredAddr.getAuthority(), defaultHost);
+    return URI.create(scheme + "://" + authority);
+  }
+
+  /**
+   * Determine whether HTTP or HTTPS should be used to connect to the remote
+   * server. Currently the client only connects to the server via HTTPS if the
+   * policy is set to HTTPS_ONLY.
+   *
+   * @return the scheme (HTTP / HTTPS)
+   */
+  public static String getHttpClientScheme(Configuration conf) {
+    HttpConfig.Policy policy = DFSUtil.getHttpPolicy(conf);
+    return policy == HttpConfig.Policy.HTTPS_ONLY ? "https" : "http";
   }
-  
 
   /**
    * Substitute a default host in the case that an address has been configured
@@ -1004,8 +1039,9 @@ public class DFSUtil {
    * @return the substituted address
    * @throws IOException if it is a wildcard address and security is enabled
    */
-  public static String substituteForWildcardAddress(String configuredAddress,
-      String defaultHost) throws IOException {
+  @VisibleForTesting
+  static String substituteForWildcardAddress(String configuredAddress,
+    String defaultHost) throws IOException {
     InetSocketAddress sockAddr = NetUtils.createSocketAddr(configuredAddress);
     InetSocketAddress defaultSockAddr = NetUtils.createSocketAddr(defaultHost
         + ":0");
@@ -1539,4 +1575,67 @@ public class DFSUtil {
     }
     return ttl*1000;
   }
+
+  /**
+   * Load HTTPS-related configuration.
+   */
+  public static Configuration loadSslConfiguration(Configuration conf) {
+    Configuration sslConf = new Configuration(false);
+
+    sslConf.addResource(conf.get(
+        DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY,
+        DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_DEFAULT));
+
+    boolean requireClientAuth = conf.getBoolean(DFS_CLIENT_HTTPS_NEED_AUTH_KEY,
+        DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT);
+    sslConf.setBoolean(DFS_CLIENT_HTTPS_NEED_AUTH_KEY, requireClientAuth);
+    return sslConf;
+  }
+
+  /**
+   * Return a HttpServer.Builder that the journalnode / namenode / secondary
+   * namenode can use to initialize their HTTP / HTTPS server.
+   *
+   */
+  public static HttpServer.Builder httpServerTemplateForNNAndJN(
+      Configuration conf, final InetSocketAddress httpAddr,
+      final InetSocketAddress httpsAddr, String name, String spnegoUserNameKey,
+      String spnegoKeytabFileKey) throws IOException {
+    HttpConfig.Policy policy = getHttpPolicy(conf);
+
+    HttpServer.Builder builder = new HttpServer.Builder().setName(name)
+        .setConf(conf).setACL(new AccessControlList(conf.get(DFS_ADMIN, " ")))
+        .setSecurityEnabled(UserGroupInformation.isSecurityEnabled())
+        .setUsernameConfKey(spnegoUserNameKey)
+        .setKeytabConfKey(getSpnegoKeytabKey(conf, spnegoKeytabFileKey));
+
+    // initialize the webserver for uploading/downloading files.
+    LOG.info("Starting web server as: "
+        + SecurityUtil.getServerPrincipal(conf.get(spnegoUserNameKey),
+            httpAddr.getHostName()));
+
+    if (policy.isHttpEnabled()) {
+      if (httpAddr.getPort() == 0) {
+        builder.setFindPort(true);
+      }
+
+      URI uri = URI.create("http://" + NetUtils.getHostPortString(httpAddr));
+      builder.addEndpoint(uri);
+      LOG.info("Starting Web-server for " + name + " at: " + uri);
+    }
+
+    if (policy.isHttpsEnabled() && httpsAddr != null) {
+      Configuration sslConf = loadSslConfiguration(conf);
+      loadSslConfToHttpServerBuilder(builder, sslConf);
+
+      if (httpsAddr.getPort() == 0) {
+        builder.setFindPort(true);
+      }
+
+      URI uri = URI.create("https://" + NetUtils.getHostPortString(httpsAddr));
+      builder.addEndpoint(uri);
+      LOG.info("Starting Web-server for " + name + " at: " + uri);
+    }
+    return builder;
+  }
 }

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/BasicInetPeer.java

@@ -125,4 +125,9 @@ class BasicInetPeer implements Peer {
   public DomainSocket getDomainSocket() {
     return null;
   }
+
+  @Override
+  public boolean hasSecureChannel() {
+    return false;
+  }
 }

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/DomainPeer.java

@@ -114,4 +114,19 @@ public class DomainPeer implements Peer {
   public DomainSocket getDomainSocket() {
     return socket;
   }
+
+  @Override
+  public boolean hasSecureChannel() {
+    //
+    // Communication over domain sockets is assumed to be secure, since it
+    // doesn't pass over any network.  We also carefully control the privileges
+    // that can be used on the domain socket inode and its parent directories.
+    // See #{java.org.apache.hadoop.net.unix.DomainSocket#validateSocketPathSecurity0}
+    // for details.
+    //
+    // So unless you are running as root or the hdfs superuser, you cannot
+    // launch a man-in-the-middle attach on UNIX domain socket traffic.
+    //
+    return true;
+  }
 }

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/EncryptedPeer.java

@@ -139,4 +139,9 @@ public class EncryptedPeer implements Peer {
   public DomainSocket getDomainSocket() {
     return enclosedPeer.getDomainSocket();
   }
+
+  @Override
+  public boolean hasSecureChannel() {
+    return true;
+  }
 }

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/NioInetPeer.java

@@ -128,4 +128,9 @@ class NioInetPeer implements Peer {
   public DomainSocket getDomainSocket() {
     return null;
   }
+
+  @Override
+  public boolean hasSecureChannel() {
+    return false;
+  }
 }

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/Peer.java

@@ -112,4 +112,12 @@ public interface Peer extends Closeable {
    *                       peer, or null if there is none.
    */
   public DomainSocket getDomainSocket();
+  
+  /**
+   * Return true if the channel is secure.
+   *
+   * @return               True if our channel to this peer is not
+   *                       susceptible to man-in-the-middle attacks.
+   */
+  public boolean hasSecureChannel();
 }

+ 53 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.qjournal.client;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.MalformedURLException;
+import java.net.URI;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
 import java.util.concurrent.Callable;
@@ -84,8 +85,9 @@ public class IPCLoggerChannel implements AsyncLogger {
   
   private final String journalId;
   private final NamespaceInfo nsInfo;
-  private int httpPort = -1;
-  
+
+  private URL httpServerURL;
+
   private final IPCLoggerChannelMetrics metrics;
   
   /**
@@ -241,13 +243,12 @@ public class IPCLoggerChannel implements AsyncLogger {
   public URL buildURLToFetchLogs(long segmentTxId) {
     Preconditions.checkArgument(segmentTxId > 0,
         "Invalid segment: %s", segmentTxId);
-    Preconditions.checkState(httpPort != -1,
-        "HTTP port not set yet");
+    Preconditions.checkState(hasHttpServerEndPoint(), "No HTTP/HTTPS endpoint");
         
     try {
       String path = GetJournalEditServlet.buildPath(
           journalId, segmentTxId, nsInfo);
-      return new URL("http", addr.getHostName(), httpPort, path.toString());
+      return new URL(httpServerURL, path);
     } catch (MalformedURLException e) {
       // should never get here.
       throw new RuntimeException(e);
@@ -313,7 +314,7 @@ public class IPCLoggerChannel implements AsyncLogger {
       public GetJournalStateResponseProto call() throws IOException {
         GetJournalStateResponseProto ret =
             getProxy().getJournalState(journalId);
-        httpPort = ret.getHttpPort();
+        constructHttpServerURI(ret);
         return ret;
       }
     });
@@ -528,7 +529,7 @@ public class IPCLoggerChannel implements AsyncLogger {
             journalId, fromTxnId, forReading, inProgressOk);
         // Update the http port, since we need this to build URLs to any of the
         // returned logs.
-        httpPort = ret.getHttpPort();
+        constructHttpServerURI(ret);
         return PBHelper.convert(ret.getManifest());
       }
     });
@@ -540,10 +541,12 @@ public class IPCLoggerChannel implements AsyncLogger {
     return executor.submit(new Callable<PrepareRecoveryResponseProto>() {
       @Override
       public PrepareRecoveryResponseProto call() throws IOException {
-        if (httpPort < 0) {
-          // If the HTTP port hasn't been set yet, force an RPC call so we know
-          // what the HTTP port should be.
-          httpPort = getProxy().getJournalState(journalId).getHttpPort();
+        if (!hasHttpServerEndPoint()) {
+          // force an RPC call so we know what the HTTP port should be if it
+          // haven't done so.
+          GetJournalStateResponseProto ret = getProxy().getJournalState(
+              journalId);
+          constructHttpServerURI(ret);
         }
         return getProxy().prepareRecovery(createReqInfo(), segmentTxId);
       }
@@ -594,4 +597,43 @@ public class IPCLoggerChannel implements AsyncLogger {
         Math.max(lastCommitNanos - lastAckNanos, 0),
         TimeUnit.NANOSECONDS);
   }
+
+  private void constructHttpServerURI(GetEditLogManifestResponseProto ret) {
+    if (ret.hasFromURL()) {
+      URI uri = URI.create(ret.getFromURL());
+      httpServerURL = getHttpServerURI(uri.getScheme(), uri.getPort());
+    } else {
+      httpServerURL = getHttpServerURI("http", ret.getHttpPort());;
+    }
+  }
+
+  private void constructHttpServerURI(GetJournalStateResponseProto ret) {
+    if (ret.hasFromURL()) {
+      URI uri = URI.create(ret.getFromURL());
+      httpServerURL = getHttpServerURI(uri.getScheme(), uri.getPort());
+    } else {
+      httpServerURL = getHttpServerURI("http", ret.getHttpPort());;
+    }
+  }
+
+  /**
+   * Construct the http server based on the response.
+   *
+   * The fromURL field in the response specifies the endpoint of the http
+   * server. However, the address might not be accurate since the server can
+   * bind to multiple interfaces. Here the client plugs in the address specified
+   * in the configuration and generates the URI.
+   */
+  private URL getHttpServerURI(String scheme, int port) {
+    try {
+      return new URL(scheme, addr.getHostName(), port, "");
+    } catch (MalformedURLException e) {
+      // Unreachable
+      throw new RuntimeException(e);
+    }
+  }
+
+  private boolean hasHttpServerEndPoint() {
+   return httpServerURL != null;
+  }
 }

+ 7 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java

@@ -64,7 +64,7 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
   private JournalNodeHttpServer httpServer;
   private Map<String, Journal> journalsById = Maps.newHashMap();
   private ObjectName journalNodeInfoBeanName;
-
+  private String httpServerURI;
   private File localDir;
 
   static {
@@ -140,6 +140,8 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
     httpServer = new JournalNodeHttpServer(conf, this);
     httpServer.start();
 
+    httpServerURI = httpServer.getServerURI().toString();
+
     rpcServer = new JournalNodeRpcServer(conf, this);
     rpcServer.start();
   }
@@ -155,11 +157,14 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
     return rpcServer.getAddress();
   }
   
-
+  @Deprecated
   public InetSocketAddress getBoundHttpAddress() {
     return httpServer.getAddress();
   }
 
+  public String getHttpServerURI() {
+    return httpServerURI;
+  }
 
   /**
    * Stop the daemon with the given status code

+ 25 - 42
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeHttpServer.java

@@ -17,19 +17,12 @@
  */
 package org.apache.hadoop.hdfs.qjournal.server;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ADMIN;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_JOURNALNODE_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_JOURNALNODE_INTERNAL_SPNEGO_USER_NAME_KEY;
-
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
-import java.net.URISyntaxException;
 
 import javax.servlet.ServletContext;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -37,22 +30,15 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.authorize.AccessControlList;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
 
 /**
  * Encapsulates the HTTP server started by the Journal Service.
  */
 @InterfaceAudience.Private
 public class JournalNodeHttpServer {
-  public static final Log LOG = LogFactory.getLog(
-      JournalNodeHttpServer.class);
-
   public static final String JN_ATTRIBUTE_KEY = "localjournal";
 
   private HttpServer httpServer;
-  private int infoPort;
   private JournalNode localJournalNode;
 
   private final Configuration conf;
@@ -63,40 +49,24 @@ public class JournalNodeHttpServer {
   }
 
   void start() throws IOException {
-    final InetSocketAddress bindAddr = getAddress(conf);
-
-    // initialize the webserver for uploading/downloading files.
-    LOG.info("Starting web server as: "+ SecurityUtil.getServerPrincipal(conf
-        .get(DFS_JOURNALNODE_INTERNAL_SPNEGO_USER_NAME_KEY),
-        bindAddr.getHostName()));
-
-    int tmpInfoPort = bindAddr.getPort();
-    URI httpEndpoint;
-    try {
-      httpEndpoint = new URI("http://" + NetUtils.getHostPortString(bindAddr));
-    } catch (URISyntaxException e) {
-      throw new IOException(e);
-    }
+    final InetSocketAddress httpAddr = getAddress(conf);
 
-    httpServer = new HttpServer.Builder().setName("journal")
-        .addEndpoint(httpEndpoint)
-        .setFindPort(tmpInfoPort == 0).setConf(conf).setACL(
-            new AccessControlList(conf.get(DFS_ADMIN, " ")))
-        .setSecurityEnabled(UserGroupInformation.isSecurityEnabled())
-        .setUsernameConfKey(
-            DFS_JOURNALNODE_INTERNAL_SPNEGO_USER_NAME_KEY)
-        .setKeytabConfKey(DFSUtil.getSpnegoKeytabKey(conf,
-            DFS_JOURNALNODE_KEYTAB_FILE_KEY)).build();
+    final String httpsAddrString = conf.get(
+        DFSConfigKeys.DFS_JOURNALNODE_HTTPS_ADDRESS_KEY,
+        DFSConfigKeys.DFS_JOURNALNODE_HTTPS_ADDRESS_DEFAULT);
+    InetSocketAddress httpsAddr = NetUtils.createSocketAddr(httpsAddrString);
+
+    HttpServer.Builder builder = DFSUtil.httpServerTemplateForNNAndJN(conf,
+        httpAddr, httpsAddr, "journal",
+        DFSConfigKeys.DFS_JOURNALNODE_INTERNAL_SPNEGO_USER_NAME_KEY,
+        DFSConfigKeys.DFS_JOURNALNODE_KEYTAB_FILE_KEY);
+
+    httpServer = builder.build();
     httpServer.setAttribute(JN_ATTRIBUTE_KEY, localJournalNode);
     httpServer.setAttribute(JspHelper.CURRENT_CONF, conf);
     httpServer.addInternalServlet("getJournal", "/getJournal",
         GetJournalEditServlet.class, true);
     httpServer.start();
-
-    // The web-server port can be ephemeral... ensure we have the correct info
-    infoPort = httpServer.getConnectorAddress(0).getPort();
-
-    LOG.info("Journal Web-server up at: " + bindAddr + ":" + infoPort);
   }
 
   void stop() throws IOException {
@@ -112,12 +82,25 @@ public class JournalNodeHttpServer {
   /**
    * Return the actual address bound to by the running server.
    */
+  @Deprecated
   public InetSocketAddress getAddress() {
     InetSocketAddress addr = httpServer.getConnectorAddress(0);
     assert addr.getPort() != 0;
     return addr;
   }
 
+  /**
+   * Return the URI that locates the HTTP server.
+   */
+  URI getServerURI() {
+    // getHttpClientScheme() only returns https for HTTPS_ONLY policy. This
+    // matches the behavior that the first connector is a HTTPS connector only
+    // for HTTPS_ONLY policy.
+    InetSocketAddress addr = httpServer.getConnectorAddress(0);
+    return URI.create(DFSUtil.getHttpClientScheme(conf) + "://"
+        + NetUtils.getHostPortString(addr));
+  }
+
   private static InetSocketAddress getAddress(Configuration conf) {
     String addr = conf.get(DFSConfigKeys.DFS_JOURNALNODE_HTTP_ADDRESS_KEY,
         DFSConfigKeys.DFS_JOURNALNODE_HTTP_ADDRESS_DEFAULT);

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java

@@ -115,6 +115,7 @@ class JournalNodeRpcServer implements QJournalProtocol {
     return jn.getOrCreateJournal(journalId).isFormatted();
   }
 
+  @SuppressWarnings("deprecation")
   @Override
   public GetJournalStateResponseProto getJournalState(String journalId)
         throws IOException {
@@ -122,6 +123,7 @@ class JournalNodeRpcServer implements QJournalProtocol {
     return GetJournalStateResponseProto.newBuilder()
         .setLastPromisedEpoch(epoch)
         .setHttpPort(jn.getBoundHttpAddress().getPort())
+        .setFromURL(jn.getHttpServerURI())
         .build();
   }
 
@@ -173,6 +175,7 @@ class JournalNodeRpcServer implements QJournalProtocol {
       .purgeLogsOlderThan(reqInfo, minTxIdToKeep);
   }
 
+  @SuppressWarnings("deprecation")
   @Override
   public GetEditLogManifestResponseProto getEditLogManifest(String jid,
       long sinceTxId, boolean forReading, boolean inProgressOk)
@@ -184,6 +187,7 @@ class JournalNodeRpcServer implements QJournalProtocol {
     return GetEditLogManifestResponseProto.newBuilder()
         .setManifest(PBHelper.convert(manifest))
         .setHttpPort(jn.getBoundHttpAddress().getPort())
+        .setFromURL(jn.getHttpServerURI())
         .build();
   }
 

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

@@ -339,12 +339,7 @@ public class DataNode extends Configured
       InetSocketAddress secInfoSocAddr = NetUtils.createSocketAddr(conf.get(
           DFS_DATANODE_HTTPS_ADDRESS_KEY, infoHost + ":" + 0));
 
-      Configuration sslConf = new Configuration(false);
-      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));
+      Configuration sslConf = DFSUtil.loadSslConfiguration(conf);
       DFSUtil.loadSslConfToHttpServerBuilder(builder, sslConf);
 
       int port = secInfoSocAddr.getPort();

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

@@ -162,7 +162,7 @@ class DataXceiver extends Receiver implements Runnable {
     try {
       peer.setWriteTimeout(datanode.getDnConf().socketWriteTimeout);
       InputStream input = socketIn;
-      if (dnConf.encryptDataTransfer) {
+      if ((!peer.hasSecureChannel()) && dnConf.encryptDataTransfer) {
         IOStreamPair encryptedStreams = null;
         try {
           encryptedStreams = DataTransferEncryptor.getEncryptedStreams(socketOut,

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

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.SocketTimeoutException;
+import java.net.URL;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
@@ -79,7 +80,7 @@ public class BackupNode extends NameNode {
   /** Name-node RPC address */
   String nnRpcAddress;
   /** Name-node HTTP address */
-  String nnHttpAddress;
+  URL nnHttpAddress;
   /** Checkpoint manager */
   Checkpointer checkpointManager;
   
@@ -313,7 +314,8 @@ public class BackupNode extends NameNode {
         NamenodeProtocol.class, UserGroupInformation.getCurrentUser(),
         true).getProxy();
     this.nnRpcAddress = NetUtils.getHostPortString(nnAddress);
-    this.nnHttpAddress = NetUtils.getHostPortString(super.getHttpServerAddress(conf));
+    this.nnHttpAddress = DFSUtil.getInfoServer(nnAddress, conf,
+        DFSUtil.getHttpClientScheme(conf)).toURL();
     // get version and id info from the name-node
     NamespaceInfo nsInfo = null;
     while(!isStopRequested()) {

+ 12 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java

@@ -24,11 +24,14 @@ import static org.apache.hadoop.util.Time.now;
 import java.io.File;
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.net.URL;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
@@ -61,6 +64,7 @@ class Checkpointer extends Daemon {
   private String infoBindAddress;
 
   private CheckpointConf checkpointConf;
+  private final Configuration conf;
 
   private BackupImage getFSImage() {
     return (BackupImage)backupNode.getFSImage();
@@ -74,6 +78,7 @@ class Checkpointer extends Daemon {
    * Create a connection to the primary namenode.
    */
   Checkpointer(Configuration conf, BackupNode bnNode)  throws IOException {
+    this.conf = conf;
     this.backupNode = bnNode;
     try {
       initialize(conf);
@@ -274,10 +279,15 @@ class Checkpointer extends Daemon {
         + " New Image Size: " + imageSize);
   }
 
-  private InetSocketAddress getImageListenAddress() {
+  private URL getImageListenAddress() {
     InetSocketAddress httpSocAddr = backupNode.getHttpAddress();
     int httpPort = httpSocAddr.getPort();
-    return new InetSocketAddress(infoBindAddress, httpPort);
+    try {
+      return new URL(DFSUtil.getHttpClientScheme(conf) + "://" + infoBindAddress + ":" + httpPort);
+    } catch (MalformedURLException e) {
+      // Unreachable
+      throw new RuntimeException(e);
+    }
   }
 
   static void rollForwardByApplyingLogs(

+ 20 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java

@@ -21,6 +21,7 @@ import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.net.InetSocketAddress;
+import java.net.URI;
 import java.net.URL;
 import java.net.URLConnection;
 import java.util.ArrayList;
@@ -41,7 +42,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil.ConfiguredNNAddress;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
-import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.util.StringUtils;
 import org.codehaus.jackson.JsonNode;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -272,12 +272,13 @@ class ClusterJspHelper {
   static class NamenodeMXBeanHelper {
     private static final ObjectMapper mapper = new ObjectMapper();
     private final String host;
-    private final String httpAddress;
+    private final URI httpAddress;
     
     NamenodeMXBeanHelper(InetSocketAddress addr, Configuration conf)
         throws IOException, MalformedObjectNameException {
       this.host = addr.getHostName();
-      this.httpAddress = DFSUtil.getInfoServer(addr, conf, false);
+      this.httpAddress = DFSUtil.getInfoServer(addr, conf,
+          DFSUtil.getHttpClientScheme(conf));
     }
     
     /** Get the map corresponding to the JSON string */
@@ -356,7 +357,7 @@ class ClusterJspHelper {
       nn.blocksCount = getProperty(props, "TotalBlocks").getLongValue();
       nn.missingBlocksCount = getProperty(props, "NumberOfMissingBlocks")
           .getLongValue();
-      nn.httpAddress = httpAddress;
+      nn.httpAddress = httpAddress.toURL();
       getLiveNodeCount(getProperty(props, "LiveNodes").getValueAsText(), nn);
       getDeadNodeCount(getProperty(props, "DeadNodes").getValueAsText(), nn);
       nn.softwareVersion = getProperty(props, "SoftwareVersion").getTextValue();
@@ -591,12 +592,14 @@ class ClusterJspHelper {
         toXmlItemBlock(doc, "Blocks", Long.toString(nn.blocksCount));
         toXmlItemBlock(doc, "Missing Blocks",
             Long.toString(nn.missingBlocksCount));
-        toXmlItemBlockWithLink(doc, nn.liveDatanodeCount + " (" +
-          nn.liveDecomCount + ")", nn.httpAddress+"/dfsnodelist.jsp?whatNodes=LIVE",
-          "Live Datanode (Decommissioned)");
-        toXmlItemBlockWithLink(doc, nn.deadDatanodeCount + " (" +
-          nn.deadDecomCount + ")", nn.httpAddress+"/dfsnodelist.jsp?whatNodes=DEAD"
-          , "Dead Datanode (Decommissioned)");
+        toXmlItemBlockWithLink(doc, nn.liveDatanodeCount + " ("
+            + nn.liveDecomCount + ")", new URL(nn.httpAddress,
+            "/dfsnodelist.jsp?whatNodes=LIVE"),
+            "Live Datanode (Decommissioned)");
+        toXmlItemBlockWithLink(doc, nn.deadDatanodeCount + " ("
+            + nn.deadDecomCount + ")", new URL(nn.httpAddress,
+            "/dfsnodelist.jsp?whatNodes=DEAD"),
+            "Dead Datanode (Decommissioned)");
         toXmlItemBlock(doc, "Software Version", nn.softwareVersion);
         doc.endTag(); // node
       }
@@ -625,7 +628,7 @@ class ClusterJspHelper {
     int liveDecomCount = 0;
     int deadDatanodeCount = 0;
     int deadDecomCount = 0;
-    String httpAddress = null;
+    URL httpAddress = null;
     String softwareVersion = "";
   }
 
@@ -763,7 +766,8 @@ class ClusterJspHelper {
                 .equals(DecommissionStates.UNKNOWN.toString()))) {
           doc.startTag("node");
           // dn
-          toXmlItemBlockWithLink(doc, dnhost, (dnhost+":"+httpPort),"DataNode");
+          toXmlItemBlockWithLink(doc, dnhost, new URL("http", dnhost, httpPort,
+              ""), "DataNode");
 
           // overall status first
           toXmlItemBlock(doc, OVERALL_STATUS, overallStatus);
@@ -823,11 +827,11 @@ class ClusterJspHelper {
    * link="http://hostname:50070" />
    */
   private static void toXmlItemBlockWithLink(XMLOutputter doc, String value,
-      String url, String label) throws IOException {
+      URL url, String label) throws IOException {
     doc.startTag("item");
     doc.attribute("label", label);
     doc.attribute("value", value);
-    doc.attribute("link", "///" + url);
+    doc.attribute("link", url.toString());
     doc.endTag(); // item
   }
 
@@ -885,7 +889,7 @@ class ClusterJspHelper {
     return out.toString();
   }
 
-  private static String queryMbean(String httpAddress, Configuration conf) 
+  private static String queryMbean(URI httpAddress, Configuration conf)
     throws IOException {
     /**
      * Although the other namenode might support HTTPS, it is fundamentally
@@ -896,7 +900,7 @@ class ClusterJspHelper {
      *
      * As a result, we just hard code the connection as an HTTP connection.
      */
-    URL url = new URL("http://" + httpAddress + JMX_QRY);
+    URL url = new URL(httpAddress.toURL(), JMX_QRY);
     return readOutput(url);
   }
   /**

+ 2 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -55,7 +55,6 @@ import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
@@ -697,11 +696,9 @@ public class FSImageFormat {
           modificationTime, atime, blocks, replication, blockSize);
       if (underConstruction) {
         file.toUnderConstruction(clientName, clientMachine, null);
-        return fileDiffs == null ? file : new INodeFileWithSnapshot(file,
-            fileDiffs);
+          return fileDiffs == null ? file : new INodeFile(file, fileDiffs);
       } else {
-        return fileDiffs == null ? file : 
-          new INodeFileWithSnapshot(file, fileDiffs);
+          return fileDiffs == null ? file : new INodeFile(file, fileDiffs);
       }
     } else if (numBlocks == -1) {
       //directory

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

@@ -202,7 +202,6 @@ import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable.SnapshotDiffInfo;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
@@ -1765,7 +1764,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       throw new HadoopIllegalArgumentException("concat: target file "
           + target + " is empty");
     }
-    if (trgInode instanceof INodeFileWithSnapshot) {
+    if (trgInode.isWithSnapshot()) {
       throw new HadoopIllegalArgumentException("concat: target file "
           + target + " is in a snapshot");
     }

+ 10 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java

@@ -21,6 +21,7 @@ import java.security.PrivilegedExceptionAction;
 import java.util.*;
 import java.io.*;
 import java.net.InetSocketAddress;
+import java.net.URL;
 
 import javax.servlet.ServletContext;
 import javax.servlet.ServletException;
@@ -31,10 +32,8 @@ import javax.servlet.http.HttpServletResponse;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -87,8 +86,8 @@ public class GetImageServlet extends HttpServlet {
       ServletContext context = getServletContext();
       final FSImage nnImage = NameNodeHttpServer.getFsImageFromContext(context);
       final GetImageParams parsedParams = new GetImageParams(request, response);
-      final Configuration conf = 
-        (Configuration)getServletContext().getAttribute(JspHelper.CURRENT_CONF);
+      final Configuration conf = (Configuration) context
+          .getAttribute(JspHelper.CURRENT_CONF);
       
       if (UserGroupInformation.isSecurityEnabled() && 
           !isValidRequestor(context, request.getUserPrincipal().getName(), conf)) {
@@ -163,7 +162,7 @@ public class GetImageServlet extends HttpServlet {
               // issue a HTTP get request to download the new fsimage 
               MD5Hash downloadImageDigest =
                 TransferFsImage.downloadImageToStorage(
-                        parsedParams.getInfoServer(), txid,
+                        parsedParams.getInfoServer(conf), txid,
                         nnImage.getStorage(), true);
               nnImage.saveDigestAndRenameCheckpointImage(txid, downloadImageDigest);
               
@@ -309,7 +308,9 @@ public class GetImageServlet extends HttpServlet {
   }
   
   static String getParamStringToPutImage(long txid,
-      InetSocketAddress imageListenAddress, Storage storage) {
+      URL url, Storage storage) {
+    InetSocketAddress imageListenAddress = NetUtils.createSocketAddr(url
+        .getAuthority());
     String machine = !imageListenAddress.isUnresolved()
         && imageListenAddress.getAddress().isAnyLocalAddress() ? null
         : imageListenAddress.getHostName();
@@ -419,11 +420,11 @@ public class GetImageServlet extends HttpServlet {
       return isPutImage;
     }
     
-    String getInfoServer() throws IOException{
+    URL getInfoServer(Configuration conf) throws IOException {
       if (machineName == null || remoteport == 0) {
-        throw new IOException ("MachineName and port undefined");
+        throw new IOException("MachineName and port undefined");
       }
-      return machineName + ":" + remoteport;
+      return new URL(DFSUtil.getHttpClientScheme(conf), machineName, remoteport, "");
     }
     
     boolean shouldFetchLatest() {

+ 0 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -34,7 +34,6 @@ import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
@@ -322,23 +321,6 @@ public class INodeDirectory extends INodeWithAdditionalFields
     replaceChild(oldChild, ref, null);
     return ref;
   }
-  
-  private void replaceChildFile(final INodeFile oldChild,
-      final INodeFile newChild, final INodeMap inodeMap) {
-    replaceChild(oldChild, newChild, inodeMap);
-    oldChild.clear();
-    newChild.updateBlockCollection();
-  }
-
-  /** Replace a child {@link INodeFile} with an {@link INodeFileWithSnapshot}. */
-  INodeFileWithSnapshot replaceChild4INodeFileWithSnapshot(
-      final INodeFile child, final INodeMap inodeMap) {
-    Preconditions.checkArgument(!(child instanceof INodeFileWithSnapshot),
-        "Child file is already an INodeFileWithSnapshot, child=" + child);
-    final INodeFileWithSnapshot newChild = new INodeFileWithSnapshot(child);
-    replaceChildFile(child, newChild, inodeMap);
-    return newChild;
-  }
 
   @Override
   public INodeDirectory recordModification(Snapshot latest,

+ 121 - 44
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -34,7 +34,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -140,24 +140,11 @@ public class INodeFile extends INodeWithAdditionalFields
     this.blocks = that.blocks;
     this.headFeature = that.headFeature;
   }
-
-  /**
-   * If the inode contains a {@link FileUnderConstructionFeature}, return it;
-   * otherwise, return null.
-   */
-  public final FileUnderConstructionFeature getFileUnderConstructionFeature() {
-    for (Feature f = this.headFeature; f != null; f = f.nextFeature) {
-      if (f instanceof FileUnderConstructionFeature) {
-        return (FileUnderConstructionFeature) f;
-      }
-    }
-    return null;
-  }
-
-  /** Is this file under construction? */
-  @Override // BlockCollection
-  public boolean isUnderConstruction() {
-    return getFileUnderConstructionFeature() != null;
+  
+  public INodeFile(INodeFile that, FileDiffList diffs) {
+    this(that);
+    Preconditions.checkArgument(!that.isWithSnapshot());
+    this.addSnapshotFeature(diffs);
   }
 
   private void addFeature(Feature f) {
@@ -182,6 +169,25 @@ public class INodeFile extends INodeWithAdditionalFields
 
   /* Start of Under-Construction Feature */
 
+  /**
+   * If the inode contains a {@link FileUnderConstructionFeature}, return it;
+   * otherwise, return null.
+   */
+  public final FileUnderConstructionFeature getFileUnderConstructionFeature() {
+    for (Feature f = this.headFeature; f != null; f = f.nextFeature) {
+      if (f instanceof FileUnderConstructionFeature) {
+        return (FileUnderConstructionFeature) f;
+      }
+    }
+    return null;
+  }
+
+  /** Is this file under construction? */
+  @Override // BlockCollection
+  public boolean isUnderConstruction() {
+    return getFileUnderConstructionFeature() != null;
+  }
+
   /** Convert this file to an {@link INodeFileUnderConstruction}. */
   INodeFile toUnderConstruction(String clientName, String clientMachine,
       DatanodeDescriptor clientNode) {
@@ -266,24 +272,75 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   /* End of Under-Construction Feature */
+  
+  /* Start of Snapshot Feature */
+
+  private FileWithSnapshotFeature addSnapshotFeature(FileDiffList diffs) {
+    FileWithSnapshotFeature sf = new FileWithSnapshotFeature(diffs);
+    this.addFeature(sf);
+    return sf;
+  }
+  
+  /**
+   * If feature list contains a {@link FileWithSnapshotFeature}, return it;
+   * otherwise, return null.
+   */
+  public final FileWithSnapshotFeature getFileWithSnapshotFeature() {
+    for (Feature f = headFeature; f != null; f = f.nextFeature) {
+      if (f instanceof FileWithSnapshotFeature) {
+        return (FileWithSnapshotFeature) f;
+      }
+    }
+    return null;
+  }
+
+  /** Is this file has the snapshot feature? */
+  public final boolean isWithSnapshot() {
+    return getFileWithSnapshotFeature() != null;
+  }
+    
+  @Override
+  public String toDetailString() {
+    FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
+    return super.toDetailString() + (sf == null ? "" : sf.getDetailedString()); 
+  }
 
   @Override
   public INodeFileAttributes getSnapshotINode(final Snapshot snapshot) {
-    return this;
+    FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
+    if (sf != null) {
+      return sf.getSnapshotINode(this, snapshot);
+    } else {
+      return this;
+    }
   }
 
   @Override
   public INodeFile recordModification(final Snapshot latest,
       final INodeMap inodeMap) throws QuotaExceededException {
     if (isInLatestSnapshot(latest)) {
-      INodeFileWithSnapshot newFile = getParent()
-          .replaceChild4INodeFileWithSnapshot(this, inodeMap)
-          .recordModification(latest, inodeMap);
-      return newFile;
-    } else {
-      return this;
+      // the file is in snapshot, create a snapshot feature if it does not have
+      FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
+      if (sf == null) {
+        sf = addSnapshotFeature(null);
+      }
+      // record self in the diff list if necessary
+      if (!shouldRecordInSrcSnapshot(latest)) {
+        sf.getDiffs().saveSelf2Snapshot(latest, this, null);
+      }
+    }
+    return this;
+  }
+  
+  public FileDiffList getDiffs() {
+    FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
+    if (sf != null) {
+      return sf.getDiffs();
     }
+    return null;
   }
+  
+  /* End of Snapshot Feature */
 
   /** @return the replication factor of the file. */
   public final short getFileReplication(Snapshot snapshot) {
@@ -295,14 +352,23 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   /** The same as getFileReplication(null). */
-  @Override
+  @Override // INodeFileAttributes
   public final short getFileReplication() {
     return getFileReplication(null);
   }
 
-  @Override
+  @Override // BlockCollection
   public short getBlockReplication() {
-    return getFileReplication(null);
+    short max = getFileReplication(null);
+    FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
+    if (sf != null) {
+      short maxInSnapshot = sf.getMaxBlockRepInDiffs();
+      if (sf.isCurrentFileDeleted()) {
+        return maxInSnapshot;
+      }
+      max = maxInSnapshot > max ? maxInSnapshot : max;
+    }
+    return max;
   }
 
   /** Set the replication factor of this file. */
@@ -395,12 +461,20 @@ public class INodeFile extends INodeWithAdditionalFields
       final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes, final boolean countDiffChange)
       throws QuotaExceededException {
+    FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
+    if (sf != null) {
+      return sf.cleanFile(this, snapshot, prior, collectedBlocks,
+          removedINodes, countDiffChange);
+    }
     Quota.Counts counts = Quota.Counts.newInstance();
-    if (snapshot == null && prior == null) {   
-      // this only happens when deleting the current file
+    if (snapshot == null && prior == null) {
+      // this only happens when deleting the current file and the file is not
+      // in any snapshot
       computeQuotaUsage(counts, false);
       destroyAndCollectBlocks(collectedBlocks, removedINodes);
     } else if (snapshot == null && prior != null) {
+      // when deleting the current file and the file is in snapshot, we should
+      // clean the 0-sized block if the file is UC
       FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
       if (uc != null) {
         uc.cleanZeroSizeBlock(this, collectedBlocks);
@@ -422,8 +496,9 @@ public class INodeFile extends INodeWithAdditionalFields
     clear();
     removedINodes.add(this);
     
-    if (this instanceof INodeFileWithSnapshot) {
-      ((INodeFileWithSnapshot) this).getDiffs().clear();
+    FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
+    if (sf != null) {
+      sf.clearDiffs();
     }
   }
   
@@ -438,8 +513,9 @@ public class INodeFile extends INodeWithAdditionalFields
       boolean useCache, int lastSnapshotId) {
     long nsDelta = 1;
     final long dsDelta;
-    if (this instanceof INodeFileWithSnapshot) {
-      FileDiffList fileDiffList = ((INodeFileWithSnapshot) this).getDiffs();
+    FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
+    if (sf != null) {
+      FileDiffList fileDiffList = sf.getDiffs();
       Snapshot last = fileDiffList.getLastSnapshot();
       List<FileDiff> diffs = fileDiffList.asList();
 
@@ -471,16 +547,16 @@ public class INodeFile extends INodeWithAdditionalFields
   private void computeContentSummary4Snapshot(final Content.Counts counts) {
     // file length and diskspace only counted for the latest state of the file
     // i.e. either the current state or the last snapshot
-    if (this instanceof INodeFileWithSnapshot) {
-      final INodeFileWithSnapshot withSnapshot = (INodeFileWithSnapshot) this;
-      final FileDiffList diffs = withSnapshot.getDiffs();
+    FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
+    if (sf != null) {
+      final FileDiffList diffs = sf.getDiffs();
       final int n = diffs.asList().size();
       counts.add(Content.FILE, n);
-      if (n > 0 && withSnapshot.isCurrentFileDeleted()) {
+      if (n > 0 && sf.isCurrentFileDeleted()) {
         counts.add(Content.LENGTH, diffs.getLast().getFileSize());
       }
 
-      if (withSnapshot.isCurrentFileDeleted()) {
+      if (sf.isCurrentFileDeleted()) {
         final long lastFileSize = diffs.getLast().getFileSize();
         counts.add(Content.DISKSPACE, lastFileSize * getBlockReplication());
       }
@@ -488,8 +564,8 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   private void computeContentSummary4Current(final Content.Counts counts) {
-    if (this instanceof INodeFileWithSnapshot
-        && ((INodeFileWithSnapshot) this).isCurrentFileDeleted()) {
+    FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
+    if (sf != null && sf.isCurrentFileDeleted()) {
       return;
     }
 
@@ -508,8 +584,9 @@ public class INodeFile extends INodeWithAdditionalFields
    * otherwise, get the file size from the given snapshot.
    */
   public final long computeFileSize(Snapshot snapshot) {
-    if (snapshot != null && this instanceof INodeFileWithSnapshot) {
-      final FileDiff d = ((INodeFileWithSnapshot) this).getDiffs().getDiff(
+    FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
+    if (snapshot != null && sf != null) {
+      final FileDiff d = sf.getDiffs().getDiff(
           snapshot);
       if (d != null) {
         return d.getFileSize();

+ 10 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java

@@ -27,7 +27,6 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 import com.google.common.base.Preconditions;
@@ -102,9 +101,8 @@ public abstract class INodeReference extends INode {
     }
     if (wn != null) {
       INode referred = wc.getReferredINode();
-      if (referred instanceof INodeFileWithSnapshot) {
-        return ((INodeFileWithSnapshot) referred).getDiffs().getPrior(
-            wn.lastSnapshotId);
+      if (referred.isFile() && referred.asFile().isWithSnapshot()) {
+        return referred.asFile().getDiffs().getPrior(wn.lastSnapshotId);
       } else if (referred instanceof INodeDirectoryWithSnapshot) { 
         return ((INodeDirectoryWithSnapshot) referred).getDiffs().getPrior(
             wn.lastSnapshotId);
@@ -547,9 +545,8 @@ public abstract class INodeReference extends INode {
     private Snapshot getSelfSnapshot() {
       INode referred = getReferredINode().asReference().getReferredINode();
       Snapshot snapshot = null;
-      if (referred instanceof INodeFileWithSnapshot) {
-        snapshot = ((INodeFileWithSnapshot) referred).getDiffs().getPrior(
-            lastSnapshotId);
+      if (referred.isFile() && referred.asFile().isWithSnapshot()) {
+        snapshot = referred.asFile().getDiffs().getPrior(lastSnapshotId);
       } else if (referred instanceof INodeDirectoryWithSnapshot) {
         snapshot = ((INodeDirectoryWithSnapshot) referred).getDiffs().getPrior(
             lastSnapshotId);
@@ -637,12 +634,12 @@ public abstract class INodeReference extends INode {
         Snapshot snapshot = getSelfSnapshot(prior);
         
         INode referred = getReferredINode().asReference().getReferredINode();
-        if (referred instanceof INodeFileWithSnapshot) {
-          // if referred is a file, it must be a FileWithSnapshot since we did
+        if (referred.isFile() && referred.asFile().isWithSnapshot()) {
+          // if referred is a file, it must be a file with Snapshot since we did
           // recordModification before the rename
-          INodeFileWithSnapshot sfile = (INodeFileWithSnapshot) referred;
+          INodeFile file = referred.asFile();
           // make sure we mark the file as deleted
-          sfile.deleteCurrentFile();
+          file.getFileWithSnapshotFeature().deleteCurrentFile();
           try {
             // when calling cleanSubtree of the referred node, since we 
             // compute quota usage updates before calling this destroy 
@@ -671,9 +668,8 @@ public abstract class INodeReference extends INode {
       WithCount wc = (WithCount) getReferredINode().asReference();
       INode referred = wc.getReferredINode();
       Snapshot lastSnapshot = null;
-      if (referred instanceof INodeFileWithSnapshot) {
-        lastSnapshot = ((INodeFileWithSnapshot) referred).getDiffs()
-            .getLastSnapshot(); 
+      if (referred.isFile() && referred.asFile().isWithSnapshot()) {
+        lastSnapshot = referred.asFile().getDiffs().getLastSnapshot();
       } else if (referred instanceof INodeDirectoryWithSnapshot) {
         lastSnapshot = ((INodeDirectoryWithSnapshot) referred)
             .getLastSnapshot();

+ 14 - 50
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java

@@ -17,13 +17,9 @@
  */
 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;
-import java.net.URI;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -39,12 +35,12 @@ 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.hdfs.web.resources.UserParam;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AccessControlList;
 
 /**
  * Encapsulates the HTTP server started by the NameNode. 
@@ -73,7 +69,10 @@ public class NameNodeHttpServer {
 
   private void initWebHdfs(Configuration conf) throws IOException {
     if (WebHdfsFileSystem.isEnabled(conf, HttpServer.LOG)) {
-      //add SPNEGO authentication filter for webhdfs
+      // set user pattern based on configuration file
+      UserParam.setUserPattern(conf.get(DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_KEY, DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_DEFAULT));
+
+      // add SPNEGO authentication filter for webhdfs
       final String name = "SPNEGO";
       final String classname = AuthFilter.class.getName();
       final String pathSpec = WebHdfsFileSystem.PATH_PREFIX + "/*";
@@ -98,51 +97,16 @@ public class NameNodeHttpServer {
     HttpConfig.Policy policy = DFSUtil.getHttpPolicy(conf);
     final String infoHost = bindAddress.getHostName();
 
-    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));
-
-    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);
-      InetSocketAddress addr = NetUtils.createSocketAddr(httpsAddrString);
+    final InetSocketAddress httpAddr = bindAddress;
+    final String httpsAddrString = conf.get(
+        DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY,
+        DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT);
+    InetSocketAddress httpsAddr = NetUtils.createSocketAddr(httpsAddrString);
 
-      Configuration sslConf = new Configuration(false);
-
-      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 builder = DFSUtil.httpServerTemplateForNNAndJN(conf,
+        httpAddr, httpsAddr, "hdfs",
+        DFSConfigKeys.DFS_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY,
+        DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY);
 
     httpServer = builder.build();
 

+ 51 - 44
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -17,12 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ADMIN;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_METRICS_SESSION_ID_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SECONDARY_NAMENODE_USER_NAME_KEY;
 import static org.apache.hadoop.util.ExitUtil.terminate;
 
 import java.io.File;
@@ -30,6 +24,7 @@ import java.io.FilenameFilter;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
+import java.net.URL;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
@@ -69,6 +64,7 @@ import org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager.StorageP
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.ipc.RemoteException;
@@ -77,7 +73,6 @@ import org.apache.hadoop.metrics2.source.JvmMetrics;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
@@ -111,7 +106,7 @@ public class SecondaryNameNode implements Runnable {
   private final long starttime = Time.now();
   private volatile long lastCheckpointTime = 0;
 
-  private String fsName;
+  private URL fsName;
   private CheckpointStorage checkpointImage;
 
   private NamenodeProtocol namenode;
@@ -119,8 +114,7 @@ public class SecondaryNameNode implements Runnable {
   private InetSocketAddress nameNodeAddr;
   private volatile boolean shouldRun;
   private HttpServer infoServer;
-  private int infoPort;
-  private String infoBindAddress;
+  private URL imageListenURL;
 
   private Collection<URI> checkpointDirs;
   private List<URI> checkpointEditsDirs;
@@ -208,8 +202,8 @@ public class SecondaryNameNode implements Runnable {
   
   public static InetSocketAddress getHttpAddress(Configuration conf) {
     return NetUtils.createSocketAddr(conf.get(
-        DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
-        DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_DEFAULT));
+        DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
+        DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_DEFAULT));
   }
   
   /**
@@ -219,17 +213,19 @@ public class SecondaryNameNode implements Runnable {
   private void initialize(final Configuration conf,
       CommandLineOpts commandLineOpts) throws IOException {
     final InetSocketAddress infoSocAddr = getHttpAddress(conf);
-    infoBindAddress = infoSocAddr.getHostName();
+    final String infoBindAddress = infoSocAddr.getHostName();
     UserGroupInformation.setConfiguration(conf);
     if (UserGroupInformation.isSecurityEnabled()) {
-      SecurityUtil.login(conf, DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY,
-          DFS_SECONDARY_NAMENODE_USER_NAME_KEY, infoBindAddress);
+      SecurityUtil.login(conf,
+          DFSConfigKeys.DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY,
+          DFSConfigKeys.DFS_SECONDARY_NAMENODE_USER_NAME_KEY, infoBindAddress);
     }
     // initiate Java VM metrics
     DefaultMetricsSystem.initialize("SecondaryNameNode");
     JvmMetrics.create("SecondaryNameNode",
-        conf.get(DFS_METRICS_SESSION_ID_KEY), DefaultMetricsSystem.instance());
-    
+        conf.get(DFSConfigKeys.DFS_METRICS_SESSION_ID_KEY),
+        DefaultMetricsSystem.instance());
+
     // Create connection to the namenode.
     shouldRun = true;
     nameNodeAddr = NameNode.getServiceAddress(conf, true);
@@ -254,19 +250,19 @@ public class SecondaryNameNode implements Runnable {
     // Initialize other scheduling parameters from the configuration
     checkpointConf = new CheckpointConf(conf);
 
-    // initialize the webserver for uploading files.
-    int tmpInfoPort = infoSocAddr.getPort();
-    URI httpEndpoint = URI.create("http://" + NetUtils.getHostPortString(infoSocAddr));
+    final InetSocketAddress httpAddr = infoSocAddr;
+
+    final String httpsAddrString = conf.get(
+        DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTPS_ADDRESS_KEY,
+        DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTPS_ADDRESS_DEFAULT);
+    InetSocketAddress httpsAddr = NetUtils.createSocketAddr(httpsAddrString);
+
+    HttpServer.Builder builder = DFSUtil.httpServerTemplateForNNAndJN(conf,
+        httpAddr, httpsAddr, "secondary",
+        DFSConfigKeys.DFS_SECONDARY_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY,
+        DFSConfigKeys.DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY);
 
-    infoServer = new HttpServer.Builder().setName("secondary")
-        .addEndpoint(httpEndpoint)
-        .setFindPort(tmpInfoPort == 0).setConf(conf).setACL(
-            new AccessControlList(conf.get(DFS_ADMIN, " ")))
-        .setSecurityEnabled(UserGroupInformation.isSecurityEnabled())
-        .setUsernameConfKey(
-            DFSConfigKeys.DFS_SECONDARY_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY)
-        .setKeytabConfKey(DFSUtil.getSpnegoKeytabKey(conf,
-            DFSConfigKeys.DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY)).build();
+    infoServer = builder.build();
 
     infoServer.setAttribute("secondary.name.node", this);
     infoServer.setAttribute("name.system.image", checkpointImage);
@@ -276,14 +272,25 @@ public class SecondaryNameNode implements Runnable {
     infoServer.start();
 
     LOG.info("Web server init done");
+    imageListenURL = new URL(DFSUtil.getHttpClientScheme(conf) + "://"
+        + NetUtils.getHostPortString(infoServer.getConnectorAddress(0)));
 
-    // The web-server port can be ephemeral... ensure we have the correct info
-    infoPort = infoServer.getConnectorAddress(0).getPort();
+    HttpConfig.Policy policy = DFSUtil.getHttpPolicy(conf);
+    int connIdx = 0;
+    if (policy.isHttpEnabled()) {
+      InetSocketAddress httpAddress = infoServer.getConnectorAddress(connIdx++);
+      conf.set(DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
+          NetUtils.getHostPortString(httpAddress));
+    }
+
+    if (policy.isHttpsEnabled()) {
+      InetSocketAddress httpsAddress = infoServer.getConnectorAddress(connIdx);
+      conf.set(DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTPS_ADDRESS_KEY,
+          NetUtils.getHostPortString(httpsAddress));
+    }
 
-    conf.set(DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY, infoBindAddress + ":" + infoPort);
-    LOG.info("Secondary Web-server up at: " + infoBindAddress + ":" + infoPort);
-    LOG.info("Checkpoint Period   :" + checkpointConf.getPeriod() + " secs " +
-             "(" + checkpointConf.getPeriod() / 60 + " min)");
+    LOG.info("Checkpoint Period   :" + checkpointConf.getPeriod() + " secs "
+        + "(" + checkpointConf.getPeriod() / 60 + " min)");
     LOG.info("Log Size Trigger    :" + checkpointConf.getTxnCount() + " txns");
   }
 
@@ -404,7 +411,7 @@ public class SecondaryNameNode implements Runnable {
    * @throws IOException
    */
   static boolean downloadCheckpointFiles(
-      final String nnHostPort,
+      final URL nnHostPort,
       final FSImage dstImage,
       final CheckpointSignature sig,
       final RemoteEditLogManifest manifest
@@ -467,25 +474,25 @@ public class SecondaryNameNode implements Runnable {
   /**
    * Returns the Jetty server that the Namenode is listening on.
    */
-  private String getInfoServer() throws IOException {
+  private URL getInfoServer() throws IOException {
     URI fsName = FileSystem.getDefaultUri(conf);
     if (!HdfsConstants.HDFS_URI_SCHEME.equalsIgnoreCase(fsName.getScheme())) {
       throw new IOException("This is not a DFS");
     }
 
-    String configuredAddress = DFSUtil.getInfoServer(null, conf, false);
-    String address = DFSUtil.substituteForWildcardAddress(configuredAddress,
-        fsName.getHost());
-    LOG.debug("Will connect to NameNode at HTTP address: " + address);
-    return address;
+    final String scheme = DFSUtil.getHttpClientScheme(conf);
+    URI address = DFSUtil.getInfoServerWithDefaultHost(fsName.getHost(), conf,
+        scheme);
+    LOG.debug("Will connect to NameNode at " + address);
+    return address.toURL();
   }
   
   /**
    * Return the host:port of where this SecondaryNameNode is listening
    * for image transfers
    */
-  private InetSocketAddress getImageListenAddress() {
-    return new InetSocketAddress(infoBindAddress, infoPort);
+  private URL getImageListenAddress() {
+    return imageListenURL;
   }
 
   /**

+ 22 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java

@@ -17,13 +17,18 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import java.io.*;
-import java.net.*;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.HttpURLConnection;
+import java.net.InetSocketAddress;
+import java.net.URL;
 import java.security.DigestInputStream;
 import java.security.MessageDigest;
 import java.util.ArrayList;
 import java.util.List;
-import java.lang.Math;
 
 import javax.servlet.ServletOutputStream;
 import javax.servlet.ServletResponse;
@@ -41,14 +46,16 @@ import org.apache.hadoop.util.Time;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.hdfs.web.URLConnectionFactory;
 import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.util.Time;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
@@ -76,15 +83,15 @@ public class TransferFsImage {
 
   private static final Log LOG = LogFactory.getLog(TransferFsImage.class);
   
-  public static void downloadMostRecentImageToDirectory(String fsName,
+  public static void downloadMostRecentImageToDirectory(URL infoServer,
       File dir) throws IOException {
     String fileId = GetImageServlet.getParamStringForMostRecentImage();
-    getFileClient(fsName, fileId, Lists.newArrayList(dir),
+    getFileClient(infoServer, fileId, Lists.newArrayList(dir),
         null, false);
   }
 
   public static MD5Hash downloadImageToStorage(
-      String fsName, long imageTxId, Storage dstStorage, boolean needDigest)
+      URL fsName, long imageTxId, Storage dstStorage, boolean needDigest)
       throws IOException {
     String fileid = GetImageServlet.getParamStringForImage(
         imageTxId, dstStorage);
@@ -102,7 +109,7 @@ public class TransferFsImage {
     return hash;
   }
   
-  static void downloadEditsToStorage(String fsName, RemoteEditLog log,
+  static void downloadEditsToStorage(URL fsName, RemoteEditLog log,
       NNStorage dstStorage) throws IOException {
     assert log.getStartTxId() > 0 && log.getEndTxId() > 0 :
       "bad log: " + log;
@@ -156,17 +163,17 @@ public class TransferFsImage {
    * Requests that the NameNode download an image from this node.
    *
    * @param fsName the http address for the remote NN
-   * @param imageListenAddress the host/port where the local node is running an
+   * @param myNNAddress the host/port where the local node is running an
    *                           HTTPServer hosting GetImageServlet
    * @param storage the storage directory to transfer the image from
    * @param txid the transaction ID of the image to be uploaded
    */
-  public static void uploadImageFromStorage(String fsName,
-      InetSocketAddress imageListenAddress,
+  public static void uploadImageFromStorage(URL fsName,
+      URL myNNAddress,
       Storage storage, long txid) throws IOException {
     
     String fileid = GetImageServlet.getParamStringToPutImage(
-        txid, imageListenAddress, storage);
+        txid, myNNAddress, storage);
     // this doesn't directly upload an image, but rather asks the NN
     // to connect back to the 2NN to download the specified image.
     try {
@@ -244,17 +251,11 @@ public class TransferFsImage {
    *                   this storage object will be notified. 
    * @Return a digest of the received file if getChecksum is true
    */
-  static MD5Hash getFileClient(String nnHostPort,
+  static MD5Hash getFileClient(URL infoServer,
       String queryString, List<File> localPaths,
       Storage dstStorage, boolean getChecksum) throws IOException {
-
-    String str = HttpConfig.getSchemePrefix() + nnHostPort + "/getimage?" +
-        queryString;
-    LOG.info("Opening connection to " + str);
-    //
-    // open connection to remote server
-    //
-    URL url = new URL(str);
+    URL url = new URL(infoServer, "/getimage?" + queryString);
+    LOG.info("Opening connection to " + url);
     return doGetUrl(url, localPaths, dstStorage, getChecksum);
   }
   

+ 8 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java

@@ -23,6 +23,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
+import java.net.URL;
 import java.security.PrivilegedAction;
 import java.util.Collection;
 import java.util.List;
@@ -69,7 +70,7 @@ public class BootstrapStandby implements Tool, Configurable {
   private String nnId;
   private String otherNNId;
 
-  private String otherHttpAddr;
+  private URL otherHttpAddr;
   private InetSocketAddress otherIpcAddr;
   private Collection<URI> dirsToFormat;
   private List<URI> editUrisToFormat;
@@ -179,6 +180,7 @@ public class BootstrapStandby implements Tool, Configurable {
     // Check with the user before blowing away data.
     if (!Storage.confirmFormat(storage.dirIterable(null),
             force, interactive)) {
+      storage.close();
       return ERR_CODE_ALREADY_FORMATTED;
     }
     
@@ -203,7 +205,7 @@ public class BootstrapStandby implements Tool, Configurable {
 
     // Download that checkpoint into our storage directories.
     MD5Hash hash = TransferFsImage.downloadImageToStorage(
-        otherHttpAddr.toString(), imageTxId,
+        otherHttpAddr, imageTxId,
         storage, true);
     image.saveDigestAndRenameCheckpointImage(imageTxId, hash);
     return 0;
@@ -276,11 +278,10 @@ public class BootstrapStandby implements Tool, Configurable {
         "Could not determine valid IPC address for other NameNode (%s)" +
         ", got: %s", otherNNId, otherIpcAddr);
 
-    otherHttpAddr = DFSUtil.getInfoServer(null, otherNode, false);
-    otherHttpAddr = DFSUtil.substituteForWildcardAddress(otherHttpAddr,
-        otherIpcAddr.getHostName());
-    
-    
+    final String scheme = DFSUtil.getHttpClientScheme(conf);
+    otherHttpAddr = DFSUtil.getInfoServerWithDefaultHost(
+        otherIpcAddr.getHostName(), otherNode, scheme).toURL();
+
     dirsToFormat = FSNamesystem.getNamespaceDirs(conf);
     editUrisToFormat = FSNamesystem.getNamespaceEditsDirs(
         conf, false);

+ 14 - 24
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java

@@ -20,7 +20,8 @@ package org.apache.hadoop.hdfs.server.namenode.ha;
 import static org.apache.hadoop.util.Time.now;
 
 import java.io.IOException;
-import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URL;
 import java.security.PrivilegedAction;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
@@ -43,7 +44,6 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.SaveNamespaceCancelledException;
 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
 import org.apache.hadoop.hdfs.util.Canceler;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -66,8 +66,8 @@ public class StandbyCheckpointer {
   private long lastCheckpointTime;
   private final CheckpointerThread thread;
   private final ThreadFactory uploadThreadFactory;
-  private String activeNNAddress;
-  private InetSocketAddress myNNAddress;
+  private URL activeNNAddress;
+  private URL myNNAddress;
 
   private Object cancelLock = new Object();
   private Canceler canceler;
@@ -94,7 +94,7 @@ public class StandbyCheckpointer {
    */
   private void setNameNodeAddresses(Configuration conf) throws IOException {
     // Look up our own address.
-    String myAddrString = getHttpAddress(conf);
+    myNNAddress = getHttpAddress(conf);
 
     // Look up the active node's address
     Configuration confForActive = HAUtil.getConfForOtherNode(conf);
@@ -103,32 +103,22 @@ public class StandbyCheckpointer {
     // Sanity-check.
     Preconditions.checkArgument(checkAddress(activeNNAddress),
         "Bad address for active NN: %s", activeNNAddress);
-    Preconditions.checkArgument(checkAddress(myAddrString),
-        "Bad address for standby NN: %s", myAddrString);
-    myNNAddress = NetUtils.createSocketAddr(myAddrString);
+    Preconditions.checkArgument(checkAddress(myNNAddress),
+        "Bad address for standby NN: %s", myNNAddress);
   }
   
-  private String getHttpAddress(Configuration conf) throws IOException {
-    String configuredAddr = DFSUtil.getInfoServer(null, conf, false);
-    
-    // Use the hostname from the RPC address as a default, in case
-    // the HTTP address is configured to 0.0.0.0.
-    String hostnameFromRpc = NameNode.getServiceAddress(
-        conf, true).getHostName();
-    try {
-      return DFSUtil.substituteForWildcardAddress(
-          configuredAddr, hostnameFromRpc);
-    } catch (IOException e) {
-      throw new IllegalArgumentException(e);
-    }
+  private URL getHttpAddress(Configuration conf) throws IOException {
+    final String scheme = DFSUtil.getHttpClientScheme(conf);
+    String defaultHost = NameNode.getServiceAddress(conf, true).getHostName();
+    URI addr = DFSUtil.getInfoServerWithDefaultHost(defaultHost, conf, scheme);
+    return addr.toURL();
   }
   
   /**
    * Ensure that the given address is valid and has a port
    * specified.
    */
-  private boolean checkAddress(String addrStr) {
-    InetSocketAddress addr = NetUtils.createSocketAddr(addrStr);
+  private static boolean checkAddress(URL addr) {
     return addr.getPort() != 0;
   }
 
@@ -344,7 +334,7 @@ public class StandbyCheckpointer {
   }
 
   @VisibleForTesting
-  String getActiveNNAddress() {
+  URL getActiveNNAddress() {
     return activeNNAddress;
   }
 }

+ 11 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java

@@ -33,7 +33,7 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.Ref
  * The difference of an {@link INodeFile} between two snapshots.
  */
 public class FileDiff extends
-    AbstractINodeDiff<INodeFileWithSnapshot, INodeFileAttributes, FileDiff> {
+    AbstractINodeDiff<INodeFile, INodeFileAttributes, FileDiff> {
 
   /** The file size at snapshot creation time. */
   private final long fileSize;
@@ -56,11 +56,12 @@ public class FileDiff extends
   }
   
   @Override
-  Quota.Counts combinePosteriorAndCollectBlocks(
-      INodeFileWithSnapshot currentINode, FileDiff posterior,
-      BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
-    return currentINode.updateQuotaAndCollectBlocks(posterior, collectedBlocks,
-        removedINodes);
+  Quota.Counts combinePosteriorAndCollectBlocks(INodeFile currentINode,
+      FileDiff posterior, BlocksMapUpdateInfo collectedBlocks,
+      final List<INode> removedINodes) {
+    return currentINode.getFileWithSnapshotFeature()
+        .updateQuotaAndCollectBlocks(currentINode, posterior, collectedBlocks,
+            removedINodes);
   }
   
   @Override
@@ -84,9 +85,10 @@ public class FileDiff extends
   }
 
   @Override
-  Quota.Counts destroyDiffAndCollectBlocks(INodeFileWithSnapshot currentINode,
+  Quota.Counts destroyDiffAndCollectBlocks(INodeFile currentINode,
       BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
-    return currentINode.updateQuotaAndCollectBlocks(this, collectedBlocks,
-        removedINodes);
+    return currentINode.getFileWithSnapshotFeature()
+        .updateQuotaAndCollectBlocks(currentINode, this, collectedBlocks,
+            removedINodes);
   }
 }

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java

@@ -17,19 +17,20 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
 
 /** A list of FileDiffs for storing snapshot data. */
 public class FileDiffList extends
-    AbstractINodeDiffList<INodeFileWithSnapshot, INodeFileAttributes, FileDiff> {
+    AbstractINodeDiffList<INodeFile, INodeFileAttributes, FileDiff> {
   
   @Override
-  FileDiff createDiff(Snapshot snapshot, INodeFileWithSnapshot file) {
+  FileDiff createDiff(Snapshot snapshot, INodeFile file) {
     return new FileDiff(snapshot, file);
   }
   
   @Override
-  INodeFileAttributes createSnapshotCopy(INodeFileWithSnapshot currentINode) {
+  INodeFileAttributes createSnapshotCopy(INodeFile currentINode) {
     return new INodeFileAttributes.SnapshotCopy(currentINode);
   }
 }

+ 71 - 84
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithSnapshot.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java

@@ -23,128 +23,135 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
-import org.apache.hadoop.hdfs.server.namenode.INodeMap;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
 
 /**
- * Represent an {@link INodeFile} that is snapshotted.
+ * Feature for file with snapshot-related information.
  */
 @InterfaceAudience.Private
-public class INodeFileWithSnapshot extends INodeFile {
+public class FileWithSnapshotFeature extends INodeFile.Feature {
   private final FileDiffList diffs;
   private boolean isCurrentFileDeleted = false;
-
-  public INodeFileWithSnapshot(INodeFile f) {
-    this(f, f instanceof INodeFileWithSnapshot ? 
-        ((INodeFileWithSnapshot) f).getDiffs() : null);
-  }
-
-  public INodeFileWithSnapshot(INodeFile f, FileDiffList diffs) {
-    super(f);
+  
+  public FileWithSnapshotFeature(FileDiffList diffs) {
     this.diffs = diffs != null? diffs: new FileDiffList();
   }
 
-  /** Is the current file deleted? */
   public boolean isCurrentFileDeleted() {
     return isCurrentFileDeleted;
   }
   
-  /** Delete the file from the current tree */
+  /** 
+   * We need to distinguish two scenarios:
+   * 1) the file is still in the current file directory, it has been modified 
+   *    before while it is included in some snapshot
+   * 2) the file is not in the current file directory (deleted), but it is in
+   *    some snapshot, thus we still keep this inode
+   * For both scenarios the file has snapshot feature. We set 
+   * {@link #isCurrentFileDeleted} to true for 2).
+   */
   public void deleteCurrentFile() {
     isCurrentFileDeleted = true;
   }
 
-  @Override
-  public INodeFileAttributes getSnapshotINode(Snapshot snapshot) {
-    return diffs.getSnapshotINode(snapshot, this);
-  }
-
-  @Override
-  public INodeFileWithSnapshot recordModification(final Snapshot latest,
-      final INodeMap inodeMap) throws QuotaExceededException {
-    if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
-      diffs.saveSelf2Snapshot(latest, this, null);
-    }
-    return this;
+  public INodeFileAttributes getSnapshotINode(INodeFile f, Snapshot snapshot) {
+    return diffs.getSnapshotINode(snapshot, f);
   }
 
-  /** @return the file diff list. */
   public FileDiffList getDiffs() {
     return diffs;
   }
-
-  @Override
-  public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
-      final BlocksMapUpdateInfo collectedBlocks,
-      final List<INode> removedINodes, final boolean countDiffChange) 
+  
+  /** @return the max replication factor in diffs */
+  public short getMaxBlockRepInDiffs() {
+    short max = 0;
+    for(FileDiff d : getDiffs()) {
+      if (d.snapshotINode != null) {
+        final short replication = d.snapshotINode.getFileReplication();
+        if (replication > max) {
+          max = replication;
+        }
+      }
+    }
+    return max;
+  }
+  
+  public String getDetailedString() {
+    return (isCurrentFileDeleted()? "(DELETED), ": ", ") + diffs;
+  }
+  
+  public Quota.Counts cleanFile(final INodeFile file, final Snapshot snapshot,
+      Snapshot prior, final BlocksMapUpdateInfo collectedBlocks,
+      final List<INode> removedINodes, final boolean countDiffChange)
       throws QuotaExceededException {
-    if (snapshot == null) { // delete the current file
+    if (snapshot == null) {
+      // delete the current file while the file has snapshot feature
       if (!isCurrentFileDeleted()) {
-        recordModification(prior, null);
+        file.recordModification(prior, null);
         deleteCurrentFile();
       }
-      this.collectBlocksAndClear(collectedBlocks, removedINodes);
+      collectBlocksAndClear(file, collectedBlocks, removedINodes);
       return Quota.Counts.newInstance();
-    } else { // delete a snapshot
+    } else { // delete the snapshot
       prior = getDiffs().updatePrior(snapshot, prior);
-      return diffs.deleteSnapshotDiff(snapshot, prior, this, collectedBlocks,
+      return diffs.deleteSnapshotDiff(snapshot, prior, file, collectedBlocks,
           removedINodes, countDiffChange);
     }
   }
-
-  @Override
-  public String toDetailString() {
-    return super.toDetailString()
-        + (isCurrentFileDeleted()? "(DELETED), ": ", ") + diffs;
+  
+  public void clearDiffs() {
+    this.diffs.clear();
   }
   
-  /** 
-   * @return block replication, which is the max file replication among
-   *         the file and the diff list.
-   */
-  @Override
-  public short getBlockReplication() {
-    short max = isCurrentFileDeleted() ? 0 : getFileReplication();
-    for(FileDiff d : getDiffs()) {
-      if (d.snapshotINode != null) {
-        final short replication = d.snapshotINode.getFileReplication();
-        if (replication > max) {
-          max = replication;
-        }
+  public Quota.Counts updateQuotaAndCollectBlocks(INodeFile file,
+      FileDiff removed, BlocksMapUpdateInfo collectedBlocks,
+      final List<INode> removedINodes) {
+    long oldDiskspace = file.diskspaceConsumed();
+    if (removed.snapshotINode != null) {
+      short replication = removed.snapshotINode.getFileReplication();
+      short currentRepl = file.getBlockReplication();
+      if (currentRepl == 0) {
+        oldDiskspace = file.computeFileSize(true, true) * replication;
+      } else if (replication > currentRepl) {  
+        oldDiskspace = oldDiskspace / file.getBlockReplication() * replication;
       }
     }
-    return max;
+    
+    collectBlocksAndClear(file, collectedBlocks, removedINodes);
+    
+    long dsDelta = oldDiskspace - file.diskspaceConsumed();
+    return Quota.Counts.newInstance(0, dsDelta);
   }
   
   /**
    * If some blocks at the end of the block list no longer belongs to
    * any inode, collect them and update the block list.
    */
-  void collectBlocksAndClear(final BlocksMapUpdateInfo info,
-      final List<INode> removedINodes) {
+  private void collectBlocksAndClear(final INodeFile file,
+      final BlocksMapUpdateInfo info, final List<INode> removedINodes) {
     // check if everything is deleted.
     if (isCurrentFileDeleted() && getDiffs().asList().isEmpty()) {
-      destroyAndCollectBlocks(info, removedINodes);
+      file.destroyAndCollectBlocks(info, removedINodes);
       return;
     }
-
     // find max file size.
     final long max;
     if (isCurrentFileDeleted()) {
       final FileDiff last = getDiffs().getLast();
       max = last == null? 0: last.getFileSize();
     } else { 
-      max = computeFileSize();
+      max = file.computeFileSize();
     }
 
-    collectBlocksBeyondMax(max, info);
+    collectBlocksBeyondMax(file, max, info);
   }
 
-  private void collectBlocksBeyondMax(final long max,
+  private void collectBlocksBeyondMax(final INodeFile file, final long max,
       final BlocksMapUpdateInfo collectedBlocks) {
-    final BlockInfo[] oldBlocks = getBlocks();
+    final BlockInfo[] oldBlocks = file.getBlocks();
     if (oldBlocks != null) {
       //find the minimum n such that the size of the first n blocks > max
       int n = 0;
@@ -164,7 +171,7 @@ public class INodeFileWithSnapshot extends INodeFile {
         }
         
         // set new blocks
-        setBlocks(newBlocks);
+        file.setBlocks(newBlocks);
 
         // collect the blocks beyond max.  
         if (collectedBlocks != null) {
@@ -175,24 +182,4 @@ public class INodeFileWithSnapshot extends INodeFile {
       }
     }
   }
-  
-  Quota.Counts updateQuotaAndCollectBlocks(FileDiff removed,
-      BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
-    long oldDiskspace = this.diskspaceConsumed();
-    if (removed.snapshotINode != null) {
-      short replication = removed.snapshotINode.getFileReplication();
-      short currentRepl = getBlockReplication();
-      if (currentRepl == 0) {
-        oldDiskspace = computeFileSize(true, true) * replication;
-      } else if (replication > currentRepl) {  
-        oldDiskspace = oldDiskspace / getBlockReplication()
-            * replication;
-      }
-    }
-    
-    this.collectBlocksAndClear(collectedBlocks, removedINodes);
-    
-    long dsDelta = oldDiskspace - diskspaceConsumed();
-    return Quota.Counts.newInstance(0, dsDelta);
-  }
 }

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectorySnapshottable.java

@@ -34,9 +34,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
-import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
+import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.namenode.Content;
 import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
 import org.apache.hadoop.hdfs.server.namenode.INode;
@@ -432,8 +432,8 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
           parentPath.remove(parentPath.size() - 1);
         }
       }
-    } else if (node.isFile() && node.asFile() instanceof INodeFileWithSnapshot) {
-      INodeFileWithSnapshot file = (INodeFileWithSnapshot) node.asFile();
+    } else if (node.isFile() && node.asFile().isWithSnapshot()) {
+      INodeFile file = node.asFile();
       Snapshot earlierSnapshot = diffReport.isFromEarlier() ? diffReport.from
           : diffReport.to;
       Snapshot laterSnapshot = diffReport.isFromEarlier() ? diffReport.to

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

@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeMap;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
@@ -803,10 +804,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
         }
         // For DstReference node, since the node is not in the created list of
         // prior, we should treat it as regular file/dir
-      } else if (topNode.isFile()
-          && topNode.asFile() instanceof INodeFileWithSnapshot) {
-        INodeFileWithSnapshot fs = (INodeFileWithSnapshot) topNode.asFile();
-        counts.add(fs.getDiffs().deleteSnapshotDiff(post, prior, fs,
+      } else if (topNode.isFile() && topNode.asFile().isWithSnapshot()) {
+        INodeFile file = topNode.asFile();
+        counts.add(file.getDiffs().deleteSnapshotDiff(post, prior, file,
             collectedBlocks, removedINodes, countDiffChange));
       } else if (topNode.isDirectory()) {
         INodeDirectory dir = topNode.asDirectory();

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

@@ -97,8 +97,7 @@ public class SnapshotFSImageFormat {
   
   public static void saveFileDiffList(final INodeFile file,
       final DataOutput out) throws IOException {
-    saveINodeDiffs(file instanceof INodeFileWithSnapshot?
-        ((INodeFileWithSnapshot) file).getDiffs(): null, out, null);
+    saveINodeDiffs(file.getDiffs(), out, null);
   }
 
   public static FileDiffList loadFileDiffList(DataInput in,

+ 6 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.tools;
 import java.io.File;
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.net.URL;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -47,9 +48,9 @@ import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
 import org.apache.hadoop.ipc.RPC;
@@ -547,8 +548,10 @@ public class DFSAdmin extends FsShell {
    * @throws IOException
    */
   public int fetchImage(final String[] argv, final int idx) throws IOException {
-    final String infoServer = DFSUtil.getInfoServer(
-        HAUtil.getAddressOfActive(getDFS()), getConf(), false);
+    Configuration conf = getConf();
+    final URL infoServer = DFSUtil.getInfoServer(
+        HAUtil.getAddressOfActive(getDFS()), conf,
+        DFSUtil.getHttpClientScheme(conf)).toURL();
     SecurityUtil.doAsCurrentUser(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {

+ 8 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java

@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.PrintStream;
+import java.net.URI;
 import java.net.URL;
 import java.net.URLConnection;
 import java.net.URLEncoder;
@@ -37,7 +38,6 @@ import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck;
 import org.apache.hadoop.hdfs.web.URLConnectionFactory;
-import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.util.StringUtils;
@@ -227,7 +227,7 @@ public class DFSck extends Configured implements Tool {
    * @return Returns http address or null if failure.
    * @throws IOException if we can't determine the active NN address
    */
-  private String getCurrentNamenodeAddress() throws IOException {
+  private URI getCurrentNamenodeAddress() throws IOException {
     //String nnAddress = null;
     Configuration conf = getConf();
 
@@ -245,19 +245,21 @@ public class DFSck extends Configured implements Tool {
       return null;
     }
     
-    return DFSUtil.getInfoServer(HAUtil.getAddressOfActive(fs), conf, false);
+    return DFSUtil.getInfoServer(HAUtil.getAddressOfActive(fs), conf,
+        DFSUtil.getHttpClientScheme(conf));
   }
 
   private int doWork(final String[] args) throws IOException {
-    final StringBuilder url = new StringBuilder(HttpConfig.getSchemePrefix());
+    final StringBuilder url = new StringBuilder();
     
-    String namenodeAddress = getCurrentNamenodeAddress();
+    URI namenodeAddress = getCurrentNamenodeAddress();
     if (namenodeAddress == null) {
       //Error message already output in {@link #getCurrentNamenodeAddress()}
       System.err.println("DFSck exiting.");
       return 0;
     }
-    url.append(namenodeAddress);
+
+    url.append(namenodeAddress.toString());
     System.err.println("Connecting to namenode via " + url.toString());
     
     url.append("/fsck?ugi=").append(ugi.getShortUserName());

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -157,6 +157,8 @@ public class WebHdfsFileSystem extends FileSystem
       ) throws IOException {
     super.initialize(uri, conf);
     setConf(conf);
+    /** set user pattern based on configuration file */
+    UserParam.setUserPattern(conf.get(DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_KEY, DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_DEFAULT));
     connectionFactory = URLConnectionFactory
         .newDefaultURLConnectionFactory(conf);
     initializeTokenAspect();

+ 20 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserParam.java

@@ -17,8 +17,10 @@
  */
 package org.apache.hadoop.hdfs.web.resources;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_DEFAULT;
 import org.apache.hadoop.security.UserGroupInformation;
-
+import com.google.common.annotations.VisibleForTesting;
+ 
 import java.text.MessageFormat;
 import java.util.regex.Pattern;
 
@@ -29,8 +31,21 @@ public class UserParam extends StringParam {
   /** Default parameter value. */
   public static final String DEFAULT = "";
 
-  private static final Domain DOMAIN = new Domain(NAME,
-    Pattern.compile("^[A-Za-z_][A-Za-z0-9._-]*[$]?$"));
+  private static Domain domain = new Domain(NAME, Pattern.compile(DFS_WEBHDFS_USER_PATTERN_DEFAULT));
+
+  @VisibleForTesting
+  public static Domain getUserPatternDomain() {
+    return domain;
+  }
+
+  @VisibleForTesting
+  public static void setUserPatternDomain(Domain dm) {
+    domain = dm;
+  }
+
+  public static void setUserPattern(String pattern) {
+    domain = new Domain(NAME, Pattern.compile(pattern));
+  }
 
   private static String validateLength(String str) {
     if (str == null) {
@@ -50,7 +65,7 @@ public class UserParam extends StringParam {
    * @param str a string representation of the parameter value.
    */
   public UserParam(final String str) {
-    super(DOMAIN, str == null || str.equals(DEFAULT)? null : validateLength(str));
+    super(domain, str == null || str.equals(DEFAULT)? null : validateLength(str));
   }
 
   /**
@@ -64,4 +79,4 @@ public class UserParam extends StringParam {
   public String getName() {
     return NAME;
   }
-}
+}

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto

@@ -142,7 +142,9 @@ message GetJournalStateRequestProto {
 
 message GetJournalStateResponseProto {
   required uint64 lastPromisedEpoch = 1;
+  // Deprecated by fromURL
   required uint32 httpPort = 2;
+  optional string fromURL = 3;
 }
 
 /**
@@ -182,7 +184,9 @@ message GetEditLogManifestRequestProto {
 
 message GetEditLogManifestResponseProto {
   required RemoteEditLogManifestProto manifest = 1; 
+  // Deprecated by fromURL
   required uint32 httpPort = 2;
+  optional string fromURL = 3;
 
   // TODO: we should add nsinfo somewhere
   // to verify that it matches up with our expectation

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

@@ -1593,4 +1593,12 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.webhdfs.user.provider.user.pattern</name>
+  <value>^[A-Za-z_][A-Za-z0-9._-]*[$]?$</value>
+  <description>
+    Valid pattern for user and group names for webhdfs, it must be a valid java regex.
+  </description>
+</property>
+
 </configuration>

+ 13 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY;
@@ -431,20 +430,22 @@ public class TestDFSUtil {
   }
 
   @Test
-  public void testGetInfoServer() throws IOException {
+  public void testGetInfoServer() throws IOException, URISyntaxException {
     HdfsConfiguration conf = new HdfsConfiguration();
-    conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    UserGroupInformation.setConfiguration(conf);
     
-    String httpsport = DFSUtil.getInfoServer(null, conf, true);
-    assertEquals("0.0.0.0:"+DFS_NAMENODE_HTTPS_PORT_DEFAULT, httpsport);
+    URI httpsport = DFSUtil.getInfoServer(null, conf, "https");
+    assertEquals(new URI("https", null, "0.0.0.0",
+        DFS_NAMENODE_HTTPS_PORT_DEFAULT, null, null, null), httpsport);
     
-    String httpport = DFSUtil.getInfoServer(null, conf, false);
-    assertEquals("0.0.0.0:"+DFS_NAMENODE_HTTP_PORT_DEFAULT, httpport);
-    
-    String httpAddress = DFSUtil.getInfoServer(new InetSocketAddress(
-        "localhost", 8020), conf, false);
-    assertEquals("localhost:" + DFS_NAMENODE_HTTP_PORT_DEFAULT, httpAddress);
+    URI httpport = DFSUtil.getInfoServer(null, conf, "http");
+    assertEquals(new URI("http", null, "0.0.0.0",
+        DFS_NAMENODE_HTTP_PORT_DEFAULT, null, null, null), httpport);
+
+    URI httpAddress = DFSUtil.getInfoServer(new InetSocketAddress(
+        "localhost", 8020), conf, "http");
+    assertEquals(
+        URI.create("http://localhost:" + DFS_NAMENODE_HTTP_PORT_DEFAULT),
+        httpAddress);
   }
   
   @Test

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

@@ -42,6 +42,10 @@ public class TestParallelShortCircuitReadUnCached extends TestParallelReadUtil {
       new File(sockDir.getDir(), 
         "TestParallelShortCircuitReadUnCached._PORT.sock").getAbsolutePath());
     conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
+    // Enabling data transfer encryption should have no effect when using
+    // short-circuit local reads.  This is a regression test for HDFS-5353.
+    conf.setBoolean(DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY, true);
+    conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
     conf.setBoolean(DFSConfigKeys.
         DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
     conf.setBoolean(DFSConfigKeys.

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

@@ -140,6 +140,11 @@ public class TestPeerCache {
     public int hashCode() {
       return dnId.hashCode() ^ (hasDomain ? 1 : 0);
     }
+
+    @Override
+    public boolean hasSecureChannel() {
+      return false;
+    }
   }
 
   @Test

+ 52 - 34
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniJournalCluster.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.qjournal.server.JournalNode;
+import org.apache.hadoop.net.NetUtils;
 
 import com.google.common.base.Joiner;
 import com.google.common.collect.Lists;
@@ -66,11 +67,21 @@ public class MiniJournalCluster {
     }
   }
 
+  private static final class JNInfo {
+    private JournalNode node;
+    private InetSocketAddress ipcAddr;
+    private String httpServerURI;
+
+    private JNInfo(JournalNode node) {
+      this.node = node;
+      this.ipcAddr = node.getBoundIpcAddress();
+      this.httpServerURI = node.getHttpServerURI();
+    }
+  }
+
   private static final Log LOG = LogFactory.getLog(MiniJournalCluster.class);
   private File baseDir;
-  private JournalNode nodes[];
-  private InetSocketAddress ipcAddrs[];
-  private InetSocketAddress httpAddrs[];
+  private JNInfo nodes[];
   
   private MiniJournalCluster(Builder b) throws IOException {
     LOG.info("Starting MiniJournalCluster with " +
@@ -81,22 +92,19 @@ public class MiniJournalCluster {
     } else {
       this.baseDir = new File(MiniDFSCluster.getBaseDirectory());
     }
-    
-    nodes = new JournalNode[b.numJournalNodes];
-    ipcAddrs = new InetSocketAddress[b.numJournalNodes];
-    httpAddrs = new InetSocketAddress[b.numJournalNodes];
+
+    nodes = new JNInfo[b.numJournalNodes];
+
     for (int i = 0; i < b.numJournalNodes; i++) {
       if (b.format) {
         File dir = getStorageDir(i);
         LOG.debug("Fully deleting JN directory " + dir);
         FileUtil.fullyDelete(dir);
       }
-      nodes[i] = new JournalNode();
-      nodes[i].setConf(createConfForNode(b, i));
-      nodes[i].start();
-
-      ipcAddrs[i] = nodes[i].getBoundIpcAddress();
-      httpAddrs[i] = nodes[i].getBoundHttpAddress();
+      JournalNode jn = new JournalNode();
+      jn.setConf(createConfForNode(b, i));
+      jn.start();
+      nodes[i] = new JNInfo(jn);
     }
   }
 
@@ -106,8 +114,8 @@ public class MiniJournalCluster {
    */
   public URI getQuorumJournalURI(String jid) {
     List<String> addrs = Lists.newArrayList();
-    for (InetSocketAddress addr : ipcAddrs) {
-      addrs.add("127.0.0.1:" + addr.getPort());
+    for (JNInfo info : nodes) {
+      addrs.add("127.0.0.1:" + info.ipcAddr.getPort());
     }
     String addrsVal = Joiner.on(";").join(addrs);
     LOG.debug("Setting logger addresses to: " + addrsVal);
@@ -122,8 +130,8 @@ public class MiniJournalCluster {
    * Start the JournalNodes in the cluster.
    */
   public void start() throws IOException {
-    for (JournalNode jn : nodes) {
-      jn.start();
+    for (JNInfo info : nodes) {
+      info.node.start();
     }
   }
 
@@ -133,12 +141,12 @@ public class MiniJournalCluster {
    */
   public void shutdown() throws IOException {
     boolean failed = false;
-    for (JournalNode jn : nodes) {
+    for (JNInfo info : nodes) {
       try {
-        jn.stopAndJoin(0);
+        info.node.stopAndJoin(0);
       } catch (Exception e) {
         failed = true;
-        LOG.warn("Unable to stop journal node " + jn, e);
+        LOG.warn("Unable to stop journal node " + info.node, e);
       }
     }
     if (failed) {
@@ -150,8 +158,8 @@ public class MiniJournalCluster {
     Configuration conf = new Configuration(b.conf);
     File logDir = getStorageDir(idx);
     conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, logDir.toString());
-    conf.set(DFSConfigKeys.DFS_JOURNALNODE_RPC_ADDRESS_KEY, "0.0.0.0:0");
-    conf.set(DFSConfigKeys.DFS_JOURNALNODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
+    conf.set(DFSConfigKeys.DFS_JOURNALNODE_RPC_ADDRESS_KEY, "localhost:0");
+    conf.set(DFSConfigKeys.DFS_JOURNALNODE_HTTP_ADDRESS_KEY, "localhost:0");
     return conf;
   }
 
@@ -164,23 +172,33 @@ public class MiniJournalCluster {
   }
 
   public JournalNode getJournalNode(int i) {
-    return nodes[i];
+    return nodes[i].node;
   }
   
   public void restartJournalNode(int i) throws InterruptedException, IOException {
-    Configuration conf = new Configuration(nodes[i].getConf());
-    if (nodes[i].isStarted()) {
-      nodes[i].stopAndJoin(0);
+    JNInfo info = nodes[i];
+    JournalNode jn = info.node;
+    Configuration conf = new Configuration(jn.getConf());
+    if (jn.isStarted()) {
+      jn.stopAndJoin(0);
     }
     
-    conf.set(DFSConfigKeys.DFS_JOURNALNODE_RPC_ADDRESS_KEY, "127.0.0.1:" +
-        ipcAddrs[i].getPort());
-    conf.set(DFSConfigKeys.DFS_JOURNALNODE_HTTP_ADDRESS_KEY, "127.0.0.1:" +
-        httpAddrs[i].getPort());
-    
-    nodes[i] = new JournalNode();
-    nodes[i].setConf(conf);
-    nodes[i].start();
+    conf.set(DFSConfigKeys.DFS_JOURNALNODE_RPC_ADDRESS_KEY,
+        NetUtils.getHostPortString(info.ipcAddr));
+
+    final String uri = info.httpServerURI;
+    if (uri.startsWith("http://")) {
+      conf.set(DFSConfigKeys.DFS_JOURNALNODE_HTTP_ADDRESS_KEY,
+          uri.substring(("http://".length())));
+    } else if (info.httpServerURI.startsWith("https://")) {
+      conf.set(DFSConfigKeys.DFS_JOURNALNODE_HTTPS_ADDRESS_KEY,
+          uri.substring(("https://".length())));
+    }
+
+    JournalNode newJN = new JournalNode();
+    newJN.setConf(conf);
+    newJN.start();
+    info.node = newJN;
   }
 
   public int getQuorumSize() {

+ 1 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java

@@ -25,7 +25,6 @@ import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.net.HttpURLConnection;
-import java.net.InetSocketAddress;
 import java.net.URL;
 import java.util.concurrent.ExecutionException;
 
@@ -163,10 +162,7 @@ public class TestJournalNode {
   
   @Test(timeout=100000)
   public void testHttpServer() throws Exception {
-    InetSocketAddress addr = jn.getBoundHttpAddress();
-    assertTrue(addr.getPort() > 0);
-    
-    String urlRoot = "http://localhost:" + addr.getPort();
+    String urlRoot = jn.getHttpServerURI();
     
     // Check default servlets.
     String pageContents = DFSTestUtil.urlGet(new URL(urlRoot + "/jmx"));

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -34,6 +34,7 @@ import java.io.RandomAccessFile;
 import java.lang.management.ManagementFactory;
 import java.net.InetSocketAddress;
 import java.net.URI;
+import java.net.URL;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -71,7 +72,6 @@ import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
@@ -218,6 +218,7 @@ public class TestCheckpoint {
     assertTrue("Removed directory wasn't what was expected",
                listRsd.size() > 0 && listRsd.get(listRsd.size() - 1).getRoot().
                toString().indexOf("storageDirToCheck") != -1);
+    nnStorage.close();
   }
 
   /*
@@ -1947,8 +1948,9 @@ public class TestCheckpoint {
           .format(true).build();
       
       NamenodeProtocols nn = cluster.getNameNodeRpc();
-      String fsName = NetUtils.getHostPortString(
-          cluster.getNameNode().getHttpAddress());
+      URL fsName = DFSUtil.getInfoServer(
+          cluster.getNameNode().getServiceRpcAddress(), conf,
+          DFSUtil.getHttpClientScheme(conf)).toURL();
 
       // Make a finalized log on the server side. 
       nn.rollEditLog();
@@ -1980,8 +1982,7 @@ public class TestCheckpoint {
       }
 
       try {
-        InetSocketAddress fakeAddr = new InetSocketAddress(1);
-        TransferFsImage.uploadImageFromStorage(fsName, fakeAddr, dstImage, 0);
+        TransferFsImage.uploadImageFromStorage(fsName, new URL("http://localhost:1234"), dstImage, 0);
         fail("Storage info was not verified");
       } catch (IOException ioe) {
         String msg = StringUtils.stringifyException(ioe);

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java

@@ -32,7 +32,6 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -239,7 +238,7 @@ public class TestSnapshotPathINodes {
     // The last INode should be the INode for sub1
     final INode last = nodesInPath.getLastINode();
     assertEquals(last.getFullPathName(), sub1.toString());
-    assertFalse(last instanceof INodeFileWithSnapshot);
+    assertFalse(last instanceof INodeFile);
     
     String[] invalidPathComponent = {"invalidDir", "foo", ".snapshot", "bar"};
     Path invalidPath = new Path(invalidPathComponent[0]);
@@ -287,7 +286,7 @@ public class TestSnapshotPathINodes {
       // Check the INode for file1 (snapshot file)
       final INode inode = inodes[inodes.length - 1];
       assertEquals(file1.getName(), inode.getLocalName());
-      assertEquals(INodeFileWithSnapshot.class, inode.getClass());
+      assertTrue(inode.asFile().isWithSnapshot());
     }
 
     // Check the INodes for path /TestSnapshot/sub1/file1
@@ -391,6 +390,8 @@ public class TestSnapshotPathINodes {
     // The last INode should be associated with file1
     assertEquals(inodes[components.length - 1].getFullPathName(),
         file1.toString());
+    // record the modification time of the inode
+    final long modTime = inodes[inodes.length - 1].getModificationTime();
     
     // Create a snapshot for the dir, and check the inodes for the path
     // pointing to a snapshot file
@@ -414,10 +415,10 @@ public class TestSnapshotPathINodes {
     // Check the INode for snapshot of file1
     INode snapshotFileNode = ssInodes[ssInodes.length - 1]; 
     assertEquals(snapshotFileNode.getLocalName(), file1.getName());
-    assertTrue(snapshotFileNode instanceof INodeFileWithSnapshot);
+    assertTrue(snapshotFileNode.asFile().isWithSnapshot());
     // The modification time of the snapshot INode should be the same with the
     // original INode before modification
-    assertEquals(inodes[inodes.length - 1].getModificationTime(),
+    assertEquals(modTime,
         snapshotFileNode.getModificationTime(ssNodesInPath.getPathSnapshot()));
 
     // Check the INode for /TestSnapshot/sub1/file1 again
@@ -432,7 +433,6 @@ public class TestSnapshotPathINodes {
     final int last = components.length - 1;
     assertEquals(newInodes[last].getFullPathName(), file1.toString());
     // The modification time of the INode for file3 should have been changed
-    Assert.assertFalse(inodes[last].getModificationTime()
-        == newInodes[last].getModificationTime());
+    Assert.assertFalse(modTime == newInodes[last].getModificationTime());
   }
 }

+ 9 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java

@@ -34,11 +34,11 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.http.HttpServerFunctionalTest;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.Test;
@@ -66,8 +66,9 @@ public class TestTransferFsImage {
         new File("/xxxxx-does-not-exist/blah"));
        
     try {
-      String fsName = NetUtils.getHostPortString(
-          cluster.getNameNode().getHttpAddress());
+      URL fsName = DFSUtil.getInfoServer(
+          cluster.getNameNode().getServiceRpcAddress(), conf,
+          DFSUtil.getHttpClientScheme(conf)).toURL();
       String id = "getimage=1&txid=0";
 
       TransferFsImage.getFileClient(fsName, id, localPath, mockStorage, false);      
@@ -98,8 +99,10 @@ public class TestTransferFsImage {
         );
        
     try {
-      String fsName = NetUtils.getHostPortString(
-          cluster.getNameNode().getHttpAddress());
+      URL fsName = DFSUtil.getInfoServer(
+          cluster.getNameNode().getServiceRpcAddress(), conf,
+          DFSUtil.getHttpClientScheme(conf)).toURL();
+
       String id = "getimage=1&txid=0";
 
       TransferFsImage.getFileClient(fsName, id, localPaths, mockStorage, false);      
@@ -123,7 +126,7 @@ public class TestTransferFsImage {
       URL serverURL = HttpServerFunctionalTest.getServerURL(testServer);
       TransferFsImage.timeout = 2000;
       try {
-        TransferFsImage.getFileClient(serverURL.getAuthority(), "txid=1", null,
+        TransferFsImage.getFileClient(serverURL, "txid=1", null,
             null, false);
         fail("TransferImage Should fail with timeout");
       } catch (SocketTimeoutException e) {

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java

@@ -24,6 +24,7 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.net.URI;
+import java.net.URL;
 import java.util.Collection;
 
 import org.apache.hadoop.conf.Configuration;
@@ -86,7 +87,8 @@ public class TestHAConfiguration {
     // 0.0.0.0, it should substitute the address from the RPC configuration
     // above.
     StandbyCheckpointer checkpointer = new StandbyCheckpointer(conf, fsn);
-    assertEquals("1.2.3.2:" + DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT,
+    assertEquals(new URL("http", "1.2.3.2",
+        DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT, ""),
         checkpointer.getActiveNNAddress());
   }
   

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java

@@ -176,7 +176,7 @@ public class TestINodeFileUnderConstructionWithSnapshot {
     dirNode = (INodeDirectorySnapshottable) fsdir.getINode(dir.toString());
     last = dirNode.getDiffs().getLast();
     Snapshot s1 = last.snapshot;
-    assertTrue(fileNode instanceof INodeFileWithSnapshot);
+    assertTrue(fileNode.isWithSnapshot());
     assertEquals(BLOCKSIZE * 3, fileNode.computeFileSize(s1));
     
     // 4. modify file --> append without closing stream --> take snapshot -->

+ 10 - 15
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java

@@ -403,8 +403,7 @@ public class TestRenameWithSnapshots {
     final Path foo_s3 = SnapshotTestHelper.getSnapshotPath(sdir1, "s3",
         "foo");
     assertFalse(hdfs.exists(foo_s3));
-    INodeFileWithSnapshot sfoo = (INodeFileWithSnapshot) fsdir.getINode(
-        newfoo.toString()).asFile();
+    INodeFile sfoo = fsdir.getINode(newfoo.toString()).asFile();
     assertEquals("s2", sfoo.getDiffs().getLastSnapshot().getRoot()
         .getLocalName());
   }
@@ -604,8 +603,7 @@ public class TestRenameWithSnapshots {
     status = hdfs.getFileStatus(foo_s2);
     assertEquals(REPL, status.getReplication());
     
-    INodeFileWithSnapshot snode = (INodeFileWithSnapshot) fsdir.getINode(
-        newfoo.toString()).asFile();
+    INodeFile snode = fsdir.getINode(newfoo.toString()).asFile();
     assertEquals(1, snode.getDiffs().asList().size());
     assertEquals("s2", snode.getDiffs().getLastSnapshot().getRoot()
         .getLocalName());
@@ -763,8 +761,7 @@ public class TestRenameWithSnapshots {
         .asDirectory();
     assertEquals(1, foo.getDiffs().asList().size());
     assertEquals("s1", foo.getLastSnapshot().getRoot().getLocalName());
-    INodeFileWithSnapshot bar1 = (INodeFileWithSnapshot) fsdir.getINode4Write(
-        bar1_dir1.toString()).asFile();
+    INodeFile bar1 = fsdir.getINode4Write(bar1_dir1.toString()).asFile();
     assertEquals(1, bar1.getDiffs().asList().size());
     assertEquals("s1", bar1.getDiffs().getLastSnapshot().getRoot()
         .getLocalName());
@@ -774,7 +771,7 @@ public class TestRenameWithSnapshots {
     INodeReference.WithCount barWithCount = (WithCount) barRef
         .getReferredINode();
     assertEquals(2, barWithCount.getReferenceCount());
-    INodeFileWithSnapshot bar = (INodeFileWithSnapshot) barWithCount.asFile();
+    INodeFile bar = barWithCount.asFile();
     assertEquals(1, bar.getDiffs().asList().size());
     assertEquals("s1", bar.getDiffs().getLastSnapshot().getRoot()
         .getLocalName());
@@ -984,8 +981,7 @@ public class TestRenameWithSnapshots {
     assertEquals("s333", fooDiffs.get(2).snapshot.getRoot().getLocalName());
     assertEquals("s22", fooDiffs.get(1).snapshot.getRoot().getLocalName());
     assertEquals("s1", fooDiffs.get(0).snapshot.getRoot().getLocalName());
-    INodeFileWithSnapshot bar1 = (INodeFileWithSnapshot) fsdir.getINode4Write(
-        bar1_dir1.toString()).asFile();
+    INodeFile bar1 = fsdir.getINode4Write(bar1_dir1.toString()).asFile();
     List<FileDiff> bar1Diffs = bar1.getDiffs().asList();
     assertEquals(3, bar1Diffs.size());
     assertEquals("s333", bar1Diffs.get(2).snapshot.getRoot().getLocalName());
@@ -997,7 +993,7 @@ public class TestRenameWithSnapshots {
     INodeReference.WithCount barWithCount = (WithCount) barRef.getReferredINode();
     // 5 references: s1, s22, s333, s2222, current tree of sdir1
     assertEquals(5, barWithCount.getReferenceCount());
-    INodeFileWithSnapshot bar = (INodeFileWithSnapshot) barWithCount.asFile();
+    INodeFile bar = barWithCount.asFile();
     List<FileDiff> barDiffs = bar.getDiffs().asList();
     assertEquals(4, barDiffs.size());
     assertEquals("s2222", barDiffs.get(3).snapshot.getRoot().getLocalName());
@@ -1047,7 +1043,7 @@ public class TestRenameWithSnapshots {
     barRef = fsdir.getINode(bar_s2222.toString()).asReference();
     barWithCount = (WithCount) barRef.getReferredINode();
     assertEquals(4, barWithCount.getReferenceCount());
-    bar = (INodeFileWithSnapshot) barWithCount.asFile();
+    bar = barWithCount.asFile();
     barDiffs = bar.getDiffs().asList();
     assertEquals(4, barDiffs.size());
     assertEquals("s2222", barDiffs.get(3).snapshot.getRoot().getLocalName());
@@ -1229,7 +1225,7 @@ public class TestRenameWithSnapshots {
       fooRef = fsdir.getINode4Write(foo2.toString());
       assertTrue(fooRef instanceof INodeReference.DstReference);
       INodeFile fooNode = fooRef.asFile();
-      assertTrue(fooNode instanceof INodeFileWithSnapshot);
+      assertTrue(fooNode.isWithSnapshot());
       assertTrue(fooNode.isUnderConstruction());
     } finally {
       if (out != null) {
@@ -1240,7 +1236,7 @@ public class TestRenameWithSnapshots {
     fooRef = fsdir.getINode4Write(foo2.toString());
     assertTrue(fooRef instanceof INodeReference.DstReference);
     INodeFile fooNode = fooRef.asFile();
-    assertTrue(fooNode instanceof INodeFileWithSnapshot);
+    assertTrue(fooNode.isWithSnapshot());
     assertFalse(fooNode.isUnderConstruction());
     
     restartClusterAndCheckImage(true);
@@ -1715,8 +1711,7 @@ public class TestRenameWithSnapshots {
     assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
     
     // bar was converted to filewithsnapshot while renaming
-    INodeFileWithSnapshot barNode = (INodeFileWithSnapshot) fsdir
-        .getINode4Write(bar.toString());
+    INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
     assertSame(barNode, children.get(0));
     assertSame(fooNode, barNode.getParent());
     List<FileDiff> barDiffList = barNode.getDiffs().asList();

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
@@ -167,7 +168,8 @@ public class TestSnapshotBlocksMap {
       Assert.assertSame(INodeFile.class, f1.getClass());
       hdfs.setReplication(file1, (short)2);
       f1 = assertBlockCollection(file1.toString(), 2, fsdir, blockmanager);
-      Assert.assertSame(INodeFileWithSnapshot.class, f1.getClass());
+      assertTrue(f1.isWithSnapshot());
+      assertFalse(f1.isUnderConstruction());
     }
     
     // Check the block information for file0

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java

@@ -277,10 +277,10 @@ public class TestSnapshotDeletion {
     TestSnapshotBlocksMap.assertBlockCollection(new Path(snapshotNoChangeDir,
         noChangeFileSCopy.getLocalName()).toString(), 1, fsdir, blockmanager);
     
-    INodeFileWithSnapshot metaChangeFile2SCopy = 
-        (INodeFileWithSnapshot) children.get(0);
+    INodeFile metaChangeFile2SCopy = children.get(0).asFile();
     assertEquals(metaChangeFile2.getName(), metaChangeFile2SCopy.getLocalName());
-    assertEquals(INodeFileWithSnapshot.class, metaChangeFile2SCopy.getClass());
+    assertTrue(metaChangeFile2SCopy.isWithSnapshot());
+    assertFalse(metaChangeFile2SCopy.isUnderConstruction());
     TestSnapshotBlocksMap.assertBlockCollection(new Path(snapshotNoChangeDir,
         metaChangeFile2SCopy.getLocalName()).toString(), 1, fsdir, blockmanager);
     
@@ -338,8 +338,9 @@ public class TestSnapshotDeletion {
     INode child = children.get(0);
     assertEquals(child.getLocalName(), metaChangeFile1.getName());
     // check snapshot copy of metaChangeFile1
-    assertEquals(INodeFileWithSnapshot.class, child.getClass());
-    INodeFileWithSnapshot metaChangeFile1SCopy = (INodeFileWithSnapshot) child;
+    INodeFile metaChangeFile1SCopy = child.asFile();
+    assertTrue(metaChangeFile1SCopy.isWithSnapshot());
+    assertFalse(metaChangeFile1SCopy.isUnderConstruction());
     assertEquals(REPLICATION_1,
         metaChangeFile1SCopy.getFileReplication(null));
     assertEquals(REPLICATION_1,

+ 28 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java

@@ -261,6 +261,34 @@ public class TestWebHDFS {
     }
   }
 
+  @Test(timeout=300000)
+  public void testNumericalUserName() throws Exception {
+    final Configuration conf = WebHdfsTestUtil.createConf();
+    conf.set(DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_KEY, "^[A-Za-z0-9_][A-Za-z0-9._-]*[$]?$");
+    final MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    try {
+      cluster.waitActive();
+      WebHdfsTestUtil.getWebHdfsFileSystem(conf, WebHdfsFileSystem.SCHEME)
+          .setPermission(new Path("/"),
+              new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
+
+      UserGroupInformation.createUserForTesting("123", new String[]{"my-group"})
+        .doAs(new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws IOException, URISyntaxException {
+            FileSystem fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
+                WebHdfsFileSystem.SCHEME);
+            Path d = new Path("/my-dir");
+            Assert.assertTrue(fs.mkdirs(d));
+            return null;
+          }
+        });
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
   /**
    * WebHdfs should be enabled by default after HDFS-5532
    * 

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java

@@ -285,4 +285,19 @@ public class TestParam {
       Assert.assertEquals(expected, computed.getValue());
     }
   }
+
+  @Test
+  public void testUserNameOkAfterResettingPattern() {
+    UserParam.Domain oldDomain = UserParam.getUserPatternDomain();
+
+    String newPattern = "^[A-Za-z0-9_][A-Za-z0-9._-]*[$]?$";
+    UserParam.setUserPattern(newPattern);
+
+    UserParam userParam = new UserParam("1x");
+    assertNotNull(userParam.getValue());
+    userParam = new UserParam("123");
+    assertNotNull(userParam.getValue());
+
+    UserParam.setUserPatternDomain(oldDomain);
+  }
 }

+ 5 - 0
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java

@@ -867,5 +867,10 @@ public class ResourceSchedulerWrapper implements ResourceScheduler,
       ApplicationAttemptId appAttemptId) {
     return scheduler.getAppResourceUsageReport(appAttemptId);
   }
+
+  @Override
+  public List<ApplicationAttemptId> getAppsInQueue(String queue) {
+    return scheduler.getAppsInQueue(queue);
+  }
 }
 

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

@@ -46,6 +46,9 @@ Release 2.4.0 - UNRELEASED
     YARN-1447. Common PB type definitions for container resizing. (Wangda Tan
     via Sandy Ryza)
 
+    YARN-1448. AM-RM protocol changes to support container resizing (Wangda Tan
+    via Sandy Ryza)
+
   IMPROVEMENTS
 
     YARN-7. Support CPU resource for DistributedShell. (Junping Du via llu)
@@ -145,6 +148,12 @@ Release 2.4.0 - UNRELEASED
 
     YARN-546. Allow disabling the Fair Scheduler event log (Sandy Ryza)
 
+    YARN-807. When querying apps by queue, iterating over all apps is
+    inefficient and limiting (Sandy Ryza)
+
+    YARN-1378. Implemented a cleaner of old finished applications from the RM
+    state-store. (Jian He via vinodkv)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -210,6 +219,12 @@ Release 2.4.0 - UNRELEASED
     YARN-1454. Fixed test failure issue with TestRMRestart. (Karthik Kambatla
     via vinodkv)
 
+    YARN-1450. Fixed test failure in TestUnmanagedAMLauncher by removing its
+    dependency on distributed-shell. (Binglin Chang via vinodkv)
+
+    YARN-1405. Fixed ResourceManager to not hang when init/start fails with an
+    exception w.r.t state-store. (Jian He via vinodkv)
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 31 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
+import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.util.Records;
 
@@ -60,12 +61,24 @@ public abstract class AllocateRequest {
       List<ResourceRequest> resourceAsk,
       List<ContainerId> containersToBeReleased,
       ResourceBlacklistRequest resourceBlacklistRequest) {
+    return newInstance(responseID, appProgress, resourceAsk,
+        containersToBeReleased, resourceBlacklistRequest, null);
+  }
+  
+  @Public
+  @Stable
+  public static AllocateRequest newInstance(int responseID, float appProgress,
+      List<ResourceRequest> resourceAsk,
+      List<ContainerId> containersToBeReleased,
+      ResourceBlacklistRequest resourceBlacklistRequest,
+      List<ContainerResourceIncreaseRequest> increaseRequests) {
     AllocateRequest allocateRequest = Records.newRecord(AllocateRequest.class);
     allocateRequest.setResponseId(responseID);
     allocateRequest.setProgress(appProgress);
     allocateRequest.setAskList(resourceAsk);
     allocateRequest.setReleaseList(containersToBeReleased);
     allocateRequest.setResourceBlacklistRequest(resourceBlacklistRequest);
+    allocateRequest.setIncreaseRequests(increaseRequests);
     return allocateRequest;
   }
   
@@ -170,4 +183,22 @@ public abstract class AllocateRequest {
   @Stable
   public abstract void setResourceBlacklistRequest(
       ResourceBlacklistRequest resourceBlacklistRequest);
+  
+  /**
+   * Get the <code>ContainerResourceIncreaseRequest</code> being sent by the
+   * <code>ApplicationMaster</code>
+   */
+  @Public
+  @Stable
+  public abstract List<ContainerResourceIncreaseRequest> getIncreaseRequests();
+  
+  /**
+   * Set the <code>ContainerResourceIncreaseRequest</code> to inform the
+   * <code>ResourceManager</code> about some container's resources need to be
+   * increased
+   */
+  @Public
+  @Stable
+  public abstract void setIncreaseRequests(
+      List<ContainerResourceIncreaseRequest> increaseRequests);
 }

+ 49 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java

@@ -28,6 +28,8 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.records.AMCommand;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease;
+import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
@@ -82,6 +84,23 @@ public abstract class AllocateResponse {
     response.setNMTokens(nmTokens);
     return response;
   }
+  
+  @Public
+  @Stable
+  public static AllocateResponse newInstance(int responseId,
+      List<ContainerStatus> completedContainers,
+      List<Container> allocatedContainers, List<NodeReport> updatedNodes,
+      Resource availResources, AMCommand command, int numClusterNodes,
+      PreemptionMessage preempt, List<NMToken> nmTokens,
+      List<ContainerResourceIncrease> increasedContainers,
+      List<ContainerResourceDecrease> decreasedContainers) {
+    AllocateResponse response = newInstance(responseId, completedContainers,
+        allocatedContainers, updatedNodes, availResources, command,
+        numClusterNodes, preempt, nmTokens);
+    response.setIncreasedContainers(increasedContainers);
+    response.setDecreasedContainers(decreasedContainers);
+    return response;
+  }
 
   /**
    * If the <code>ResourceManager</code> needs the
@@ -221,4 +240,34 @@ public abstract class AllocateResponse {
   @Private
   @Unstable
   public abstract void setNMTokens(List<NMToken> nmTokens);
+  
+  /**
+   * Get the list of newly increased containers by <code>ResourceManager</code>
+   */
+  @Public
+  @Stable
+  public abstract List<ContainerResourceIncrease> getIncreasedContainers();
+
+  /**
+   * Set the list of newly increased containers by <code>ResourceManager</code>
+   */
+  @Private
+  @Unstable
+  public abstract void setIncreasedContainers(
+      List<ContainerResourceIncrease> increasedContainers);
+
+  /**
+   * Get the list of newly decreased containers by <code>NodeManager</code>
+   */
+  @Public
+  @Stable
+  public abstract List<ContainerResourceDecrease> getDecreasedContainers();
+
+  /**
+   * Set the list of newly decreased containers by <code>NodeManager</code>
+   */
+  @Private
+  @Unstable
+  public abstract void setDecreasedContainers(
+      List<ContainerResourceDecrease> decreasedContainers);
 }

+ 10 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -342,7 +342,16 @@ public class YarnConfiguration extends Configuration {
   public static final String RM_MAX_COMPLETED_APPLICATIONS =
     RM_PREFIX + "max-completed-applications";
   public static final int DEFAULT_RM_MAX_COMPLETED_APPLICATIONS = 10000;
-  
+
+  /**
+   * The maximum number of completed applications RM state store keeps, by
+   * default equals to DEFAULT_RM_MAX_COMPLETED_APPLICATIONS
+   */
+  public static final String RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS =
+      RM_PREFIX + "state-store.max-completed-applications";
+  public static final int DEFAULT_RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS =
+      DEFAULT_RM_MAX_COMPLETED_APPLICATIONS;
+
   /** Default application name */
   public static final String DEFAULT_APPLICATION_NAME = "N/A";
 

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto

@@ -62,6 +62,7 @@ message AllocateRequestProto {
   optional ResourceBlacklistRequestProto blacklist_request = 3;
   optional int32 response_id = 4;
   optional float progress = 5;
+  repeated ContainerResourceIncreaseRequestProto increase_request = 6;
 }
 
 message NMTokenProto {
@@ -79,6 +80,8 @@ message AllocateResponseProto {
   optional int32 num_cluster_nodes = 7;
   optional PreemptionMessageProto preempt = 8;
   repeated NMTokenProto nm_tokens = 9;
+  repeated ContainerResourceIncreaseProto increased_containers = 10;
+  repeated ContainerResourceDecreaseProto decreased_containers = 11;
 }
 
 //////////////////////////////////////////////////////

+ 0 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/pom.xml

@@ -71,11 +71,6 @@
       <artifactId>hadoop-mapreduce-client-core</artifactId>
       <scope>test</scope>
     </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-applications-distributedshell</artifactId>
-      <scope>test</scope>
-    </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-server-tests</artifactId>

+ 25 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/test/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/TestUnmanagedAMLauncher.java

@@ -32,7 +32,12 @@ import junit.framework.Assert;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.client.ClientRMProxy;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.junit.AfterClass;
@@ -122,8 +127,6 @@ public class TestUnmanagedAMLauncher {
       LOG.fatal("JAVA_HOME not defined. Test not running.");
       return;
     }
-    // start dist-shell with 0 containers because container launch will fail if
-    // there are no dist cache resources.
     String[] args = {
         "--classpath",
         classpath,
@@ -132,9 +135,8 @@ public class TestUnmanagedAMLauncher {
         "--cmd",
         javaHome
             + "/bin/java -Xmx512m "
-            + "org.apache.hadoop.yarn.applications.distributedshell.ApplicationMaster "
-            + "--container_memory 128 --num_containers 1 --priority 0 "
-            + "--shell_command " + (Shell.WINDOWS ? "dir" : "ls") };
+            + TestUnmanagedAMLauncher.class.getCanonicalName()
+            + " success" };
 
     LOG.info("Initializing Launcher");
     UnmanagedAMLauncher launcher = new UnmanagedAMLauncher(new Configuration(
@@ -157,8 +159,6 @@ public class TestUnmanagedAMLauncher {
       LOG.fatal("JAVA_HOME not defined. Test not running.");
       return;
     }
-
-    // remove shell command to make dist-shell fail in initialization itself
     String[] args = {
         "--classpath",
         classpath,
@@ -167,8 +167,8 @@ public class TestUnmanagedAMLauncher {
         "--cmd",
         javaHome
             + "/bin/java -Xmx512m "
-            + "org.apache.hadoop.yarn.applications.distributedshell.ApplicationMaster "
-            + "--container_memory 128 --num_containers 1 --priority 0" };
+            + TestUnmanagedAMLauncher.class.getCanonicalName()
+            + " failure" };
 
     LOG.info("Initializing Launcher");
     UnmanagedAMLauncher launcher = new UnmanagedAMLauncher(new Configuration(
@@ -185,4 +185,19 @@ public class TestUnmanagedAMLauncher {
     }
   }
 
+  // provide main method so this class can act as AM
+  public static void main(String[] args) throws Exception {
+    if (args[0].equals("success")) {
+      ApplicationMasterProtocol client = ClientRMProxy.createRMProxy(conf,
+          ApplicationMasterProtocol.class);
+      client.registerApplicationMaster(RegisterApplicationMasterRequest
+          .newInstance(NetUtils.getHostname(), -1, ""));
+      Thread.sleep(1000);
+      client.finishApplicationMaster(FinishApplicationMasterRequest
+          .newInstance(FinalApplicationStatus.SUCCEEDED, "success", null));
+      System.exit(0);
+    } else {
+      System.exit(1);
+    }
+  }
 }

+ 86 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java

@@ -27,12 +27,15 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreaseRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceBlacklistRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceRequestPBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceBlacklistRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
@@ -49,9 +52,9 @@ public class AllocateRequestPBImpl extends AllocateRequest {
 
   private List<ResourceRequest> ask = null;
   private List<ContainerId> release = null;
+  private List<ContainerResourceIncreaseRequest> increaseRequests = null;
   private ResourceBlacklistRequest blacklistRequest = null;
   
-  
   public AllocateRequestPBImpl() {
     builder = AllocateRequestProto.newBuilder();
   }
@@ -62,7 +65,7 @@ public class AllocateRequestPBImpl extends AllocateRequest {
   }
   
   public AllocateRequestProto getProto() {
-      mergeLocalToProto();
+    mergeLocalToProto();
     proto = viaProto ? proto : builder.build();
     viaProto = true;
     return proto;
@@ -95,6 +98,9 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     if (this.release != null) {
       addReleasesToProto();
     }
+    if (this.increaseRequests != null) {
+      addIncreaseRequestsToProto();
+    }
     if (this.blacklistRequest != null) {
       builder.setBlacklistRequest(convertToProtoFormat(this.blacklistRequest));
     }
@@ -155,6 +161,23 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     this.ask.addAll(resourceRequests);
   }
   
+  @Override
+  public List<ContainerResourceIncreaseRequest> getIncreaseRequests() {
+    initIncreaseRequests();
+    return this.increaseRequests;
+  }
+
+  @Override
+  public void setIncreaseRequests(
+      List<ContainerResourceIncreaseRequest> increaseRequests) {
+    if (increaseRequests == null) {
+      return;
+    }
+    initIncreaseRequests();
+    this.increaseRequests.clear();
+    this.increaseRequests.addAll(increaseRequests);
+  }
+  
   @Override
   public ResourceBlacklistRequest getResourceBlacklistRequest() {
     AllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
@@ -223,6 +246,57 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     };
     builder.addAllAsk(iterable);
   }
+  
+  private void initIncreaseRequests() {
+    if (this.increaseRequests != null) {
+      return;
+    }
+    AllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerResourceIncreaseRequestProto> list =
+        p.getIncreaseRequestList();
+    this.increaseRequests = new ArrayList<ContainerResourceIncreaseRequest>();
+
+    for (ContainerResourceIncreaseRequestProto c : list) {
+      this.increaseRequests.add(convertFromProtoFormat(c));
+    }
+  }
+  
+  private void addIncreaseRequestsToProto() {
+    maybeInitBuilder();
+    builder.clearIncreaseRequest();
+    if (increaseRequests == null) {
+      return;
+    }
+    Iterable<ContainerResourceIncreaseRequestProto> iterable =
+        new Iterable<ContainerResourceIncreaseRequestProto>() {
+          @Override
+          public Iterator<ContainerResourceIncreaseRequestProto> iterator() {
+            return new Iterator<ContainerResourceIncreaseRequestProto>() {
+
+              Iterator<ContainerResourceIncreaseRequest> iter =
+                  increaseRequests.iterator();
+
+              @Override
+              public boolean hasNext() {
+                return iter.hasNext();
+              }
+
+              @Override
+              public ContainerResourceIncreaseRequestProto next() {
+                return convertToProtoFormat(iter.next());
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+            };
+
+          }
+        };
+    builder.addAllIncreaseRequest(iterable);
+  }
+  
   @Override
   public List<ContainerId> getReleaseList() {
     initReleases();
@@ -292,6 +366,16 @@ public class AllocateRequestPBImpl extends AllocateRequest {
   private ResourceRequestProto convertToProtoFormat(ResourceRequest t) {
     return ((ResourceRequestPBImpl)t).getProto();
   }
+  
+  private ContainerResourceIncreaseRequestPBImpl convertFromProtoFormat(
+      ContainerResourceIncreaseRequestProto p) {
+    return new ContainerResourceIncreaseRequestPBImpl(p);
+  }
+
+  private ContainerResourceIncreaseRequestProto convertToProtoFormat(
+      ContainerResourceIncreaseRequest t) {
+    return ((ContainerResourceIncreaseRequestPBImpl) t).getProto();
+  }
 
   private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
     return new ContainerIdPBImpl(p);
@@ -308,6 +392,4 @@ public class AllocateRequestPBImpl extends AllocateRequest {
   private ResourceBlacklistRequestProto convertToProtoFormat(ResourceBlacklistRequest t) {
     return ((ResourceBlacklistRequestPBImpl)t).getProto();
   }
-
-
 }  

+ 166 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java

@@ -28,12 +28,16 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.AMCommand;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease;
+import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.PreemptionMessage;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceDecreasePBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreasePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeReportPBImpl;
@@ -41,6 +45,8 @@ import org.apache.hadoop.yarn.api.records.impl.pb.PreemptionMessagePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionMessageProto;
@@ -63,6 +69,8 @@ public class AllocateResponsePBImpl extends AllocateResponse {
   private List<Container> allocatedContainers = null;
   private List<NMToken> nmTokens = null;
   private List<ContainerStatus> completedContainersStatuses = null;
+  private List<ContainerResourceIncrease> increasedContainers = null;
+  private List<ContainerResourceDecrease> decreasedContainers = null;
 
   private List<NodeReport> updatedNodes = null;
   private PreemptionMessage preempt;
@@ -108,7 +116,7 @@ public class AllocateResponsePBImpl extends AllocateResponse {
     if (this.allocatedContainers != null) {
       builder.clearAllocatedContainers();
       Iterable<ContainerProto> iterable =
-          getProtoIterable(this.allocatedContainers);
+          getContainerProtoIterable(this.allocatedContainers);
       builder.addAllAllocatedContainers(iterable);
     }
     if (nmTokens != null) {
@@ -134,6 +142,18 @@ public class AllocateResponsePBImpl extends AllocateResponse {
     if (this.preempt != null) {
       builder.setPreempt(convertToProtoFormat(this.preempt));
     }
+    if (this.increasedContainers != null) {
+      builder.clearIncreasedContainers();
+      Iterable<ContainerResourceIncreaseProto> iterable =
+          getIncreaseProtoIterable(this.increasedContainers);
+      builder.addAllIncreasedContainers(iterable);
+    }
+    if (this.decreasedContainers != null) {
+      builder.clearDecreasedContainers();
+      Iterable<ContainerResourceDecreaseProto> iterable =
+          getChangeProtoIterable(this.decreasedContainers);
+      builder.addAllDecreasedContainers(iterable);
+    }
   }
 
   private synchronized void mergeLocalToProto() {
@@ -306,6 +326,63 @@ public class AllocateResponsePBImpl extends AllocateResponse {
     this.preempt = preempt;
   }
 
+  @Override
+  public synchronized List<ContainerResourceIncrease> getIncreasedContainers() {
+    initLocalIncreasedContainerList();
+    return increasedContainers;
+  }
+
+  @Override
+  public synchronized void setIncreasedContainers(
+      List<ContainerResourceIncrease> increasedContainers) {
+    if (increasedContainers == null)
+      return;
+    initLocalIncreasedContainerList();
+    this.increasedContainers.addAll(increasedContainers);
+  }
+
+  @Override
+  public synchronized List<ContainerResourceDecrease> getDecreasedContainers() {
+    initLocalDecreasedContainerList();
+    return decreasedContainers;
+  }
+
+  @Override
+  public synchronized void setDecreasedContainers(
+      List<ContainerResourceDecrease> decreasedContainers) {
+    if (decreasedContainers == null) {
+      return;
+    }
+    initLocalDecreasedContainerList();
+    this.decreasedContainers.addAll(decreasedContainers);
+  }
+
+  private synchronized void initLocalIncreasedContainerList() {
+    if (this.increasedContainers != null) {
+      return;
+    }
+    AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerResourceIncreaseProto> list = p.getIncreasedContainersList();
+    increasedContainers = new ArrayList<ContainerResourceIncrease>();
+
+    for (ContainerResourceIncreaseProto c : list) {
+      increasedContainers.add(convertFromProtoFormat(c));
+    }
+  }
+
+  private synchronized void initLocalDecreasedContainerList() {
+    if (this.decreasedContainers != null) {
+      return;
+    }
+    AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerResourceDecreaseProto> list = p.getDecreasedContainersList();
+    decreasedContainers = new ArrayList<ContainerResourceDecrease>();
+
+    for (ContainerResourceDecreaseProto c : list) {
+      decreasedContainers.add(convertFromProtoFormat(c));
+    }
+  }
+
   // Once this is called. updatedNodes will never be null - until a getProto is
   // called.
   private synchronized void initLocalNewNodeReportList() {
@@ -348,7 +425,71 @@ public class AllocateResponsePBImpl extends AllocateResponse {
     }
   }
 
-  private synchronized Iterable<ContainerProto> getProtoIterable(
+  private synchronized Iterable<ContainerResourceIncreaseProto>
+      getIncreaseProtoIterable(
+          final List<ContainerResourceIncrease> newContainersList) {
+    maybeInitBuilder();
+    return new Iterable<ContainerResourceIncreaseProto>() {
+      @Override
+      public synchronized Iterator<ContainerResourceIncreaseProto> iterator() {
+        return new Iterator<ContainerResourceIncreaseProto>() {
+
+          Iterator<ContainerResourceIncrease> iter = newContainersList
+              .iterator();
+
+          @Override
+          public synchronized boolean hasNext() {
+            return iter.hasNext();
+          }
+
+          @Override
+          public synchronized ContainerResourceIncreaseProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+
+          @Override
+          public synchronized void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+
+      }
+    };
+  }
+
+  private synchronized Iterable<ContainerResourceDecreaseProto>
+      getChangeProtoIterable(
+          final List<ContainerResourceDecrease> newContainersList) {
+    maybeInitBuilder();
+    return new Iterable<ContainerResourceDecreaseProto>() {
+      @Override
+      public synchronized Iterator<ContainerResourceDecreaseProto> iterator() {
+        return new Iterator<ContainerResourceDecreaseProto>() {
+
+          Iterator<ContainerResourceDecrease> iter = newContainersList
+              .iterator();
+
+          @Override
+          public synchronized boolean hasNext() {
+            return iter.hasNext();
+          }
+
+          @Override
+          public synchronized ContainerResourceDecreaseProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+
+          @Override
+          public synchronized void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+
+      }
+    };
+  }
+  
+  private synchronized Iterable<ContainerProto> getContainerProtoIterable(
       final List<Container> newContainersList) {
     maybeInitBuilder();
     return new Iterable<ContainerProto>() {
@@ -467,7 +608,6 @@ public class AllocateResponsePBImpl extends AllocateResponse {
 
           }
         };
-
       }
     };
   }
@@ -486,6 +626,26 @@ public class AllocateResponsePBImpl extends AllocateResponse {
       completedContainersStatuses.add(convertFromProtoFormat(c));
     }
   }
+  
+  private synchronized ContainerResourceIncrease convertFromProtoFormat(
+      ContainerResourceIncreaseProto p) {
+    return new ContainerResourceIncreasePBImpl(p);
+  }
+
+  private synchronized ContainerResourceIncreaseProto convertToProtoFormat(
+      ContainerResourceIncrease t) {
+    return ((ContainerResourceIncreasePBImpl) t).getProto();
+  }
+
+  private synchronized ContainerResourceDecrease convertFromProtoFormat(
+      ContainerResourceDecreaseProto p) {
+    return new ContainerResourceDecreasePBImpl(p);
+  }
+
+  private synchronized ContainerResourceDecreaseProto convertToProtoFormat(
+      ContainerResourceDecrease t) {
+    return ((ContainerResourceDecreasePBImpl) t).getProto();
+  }
 
   private synchronized NodeReportPBImpl convertFromProtoFormat(
       NodeReportProto p) {
@@ -500,8 +660,9 @@ public class AllocateResponsePBImpl extends AllocateResponse {
       ContainerProto p) {
     return new ContainerPBImpl(p);
   }
-
-  private synchronized ContainerProto convertToProtoFormat(Container t) {
+  
+  private synchronized ContainerProto convertToProtoFormat(
+      Container t) {
     return ((ContainerPBImpl)t).getProto();
   }
 

+ 15 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -275,6 +275,21 @@
     <value>org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore</value>
   </property>
 
+  <property>
+    <description>The maximum number of completed applications RM state
+    store keeps, less than or equals to ${yarn.resourcemanager.max-completed-applications}.
+    By default, it equals to ${yarn.resourcemanager.max-completed-applications}.
+    This ensures that the applications kept in the state store are consistent with
+    the applications remembered in RM memory.
+    Any values larger than ${yarn.resourcemanager.max-completed-applications} will
+    be reset to ${yarn.resourcemanager.max-completed-applications}.
+    Note that this value impacts the RM recovery performance.Typically,
+    a smaller value indicates better performance on RM recovery.
+    </description>
+    <name>yarn.resourcemanager.state-store.max-completed-applications</name>
+    <value>${yarn.resourcemanager.max-completed-applications}</value>
+  </property>
+
   <property>
     <description>Host:Port of the ZooKeeper server where RM state will 
     be stored. This must be supplied when using

+ 73 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateRequest.java

@@ -0,0 +1,73 @@
+/**
+ * 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.ArrayList;
+import java.util.List;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
+import org.junit.Test;
+
+public class TestAllocateRequest {
+  @Test
+  public void testAllcoateRequestWithIncrease() {
+    List<ContainerResourceIncreaseRequest> incRequests =
+        new ArrayList<ContainerResourceIncreaseRequest>();
+    for (int i = 0; i < 3; i++) {
+      incRequests.add(ContainerResourceIncreaseRequest.newInstance(null,
+          Resource.newInstance(0, i)));
+    }
+    AllocateRequest r =
+        AllocateRequest.newInstance(123, 0f, null, null, null, incRequests);
+
+    // serde
+    AllocateRequestProto p = ((AllocateRequestPBImpl) r).getProto();
+    r = new AllocateRequestPBImpl(p);
+
+    // check value
+    Assert.assertEquals(123, r.getResponseId());
+    Assert.assertEquals(incRequests.size(), r.getIncreaseRequests().size());
+
+    for (int i = 0; i < incRequests.size(); i++) {
+      Assert.assertEquals(r.getIncreaseRequests().get(i).getCapability()
+          .getVirtualCores(), incRequests.get(i).getCapability()
+          .getVirtualCores());
+    }
+  }
+
+  @Test
+  public void testAllcoateRequestWithoutIncrease() {
+    AllocateRequest r =
+        AllocateRequest.newInstance(123, 0f, null, null, null, null);
+
+    // serde
+    AllocateRequestProto p = ((AllocateRequestPBImpl) r).getProto();
+    r = new AllocateRequestPBImpl(p);
+
+    // check value
+    Assert.assertEquals(123, r.getResponseId());
+    Assert.assertEquals(0, r.getIncreaseRequests().size());
+  }
+}

+ 114 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java

@@ -0,0 +1,114 @@
+/**
+ * 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.ArrayList;
+import java.util.List;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateResponsePBImpl;
+import org.apache.hadoop.yarn.api.records.AMCommand;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease;
+import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NMToken;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProto;
+import org.junit.Test;
+
+/**
+ * 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.
+ */
+
+public class TestAllocateResponse {
+  @Test
+  public void testAllocateResponseWithIncDecContainers() {
+    List<ContainerResourceIncrease> incContainers =
+        new ArrayList<ContainerResourceIncrease>();
+    List<ContainerResourceDecrease> decContainers =
+        new ArrayList<ContainerResourceDecrease>();
+    for (int i = 0; i < 3; i++) {
+      incContainers.add(ContainerResourceIncrease.newInstance(null,
+          Resource.newInstance(1024, i), null));
+    }
+    for (int i = 0; i < 5; i++) {
+      decContainers.add(ContainerResourceDecrease.newInstance(null,
+          Resource.newInstance(1024, i)));
+    }
+
+    AllocateResponse r =
+        AllocateResponse.newInstance(3, new ArrayList<ContainerStatus>(),
+            new ArrayList<Container>(), new ArrayList<NodeReport>(), null,
+            AMCommand.AM_RESYNC, 3, null, new ArrayList<NMToken>(),
+            incContainers, decContainers);
+
+    // serde
+    AllocateResponseProto p = ((AllocateResponsePBImpl) r).getProto();
+    r = new AllocateResponsePBImpl(p);
+
+    // check value
+    Assert
+        .assertEquals(incContainers.size(), r.getIncreasedContainers().size());
+    Assert
+        .assertEquals(decContainers.size(), r.getDecreasedContainers().size());
+
+    for (int i = 0; i < incContainers.size(); i++) {
+      Assert.assertEquals(i, r.getIncreasedContainers().get(i).getCapability()
+          .getVirtualCores());
+    }
+
+    for (int i = 0; i < decContainers.size(); i++) {
+      Assert.assertEquals(i, r.getDecreasedContainers().get(i).getCapability()
+          .getVirtualCores());
+    }
+  }
+
+  @Test
+  public void testAllocateResponseWithoutIncDecContainers() {
+    AllocateResponse r =
+        AllocateResponse.newInstance(3, new ArrayList<ContainerStatus>(),
+            new ArrayList<Container>(), new ArrayList<NodeReport>(), null,
+            AMCommand.AM_RESYNC, 3, null, new ArrayList<NMToken>(), null, null);
+
+    // serde
+    AllocateResponseProto p = ((AllocateResponsePBImpl) r).getProto();
+    r = new AllocateResponsePBImpl(p);
+
+    // check value
+    Assert.assertEquals(0, r.getIncreasedContainers().size());
+    Assert.assertEquals(0, r.getDecreasedContainers().size());
+  }
+}

+ 53 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java

@@ -24,7 +24,9 @@ import java.security.AccessControlException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.EnumSet;
+import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -67,6 +69,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
@@ -431,12 +434,51 @@ public class ClientRMService extends AbstractService implements
     LongRange start = request.getStartRange();
     LongRange finish = request.getFinishRange();
 
-    List<ApplicationReport> reports = new ArrayList<ApplicationReport>();
-    long count = 0;
-    for (RMApp application : this.rmContext.getRMApps().values()) {
-      if (++count > limit) {
-        break;
+    final Map<ApplicationId, RMApp> apps = rmContext.getRMApps();
+    Iterator<RMApp> appsIter;
+    // If the query filters by queues, we can avoid considering apps outside
+    // of those queues by asking the scheduler for the apps in those queues.
+    if (queues != null && !queues.isEmpty()) {
+      // Construct an iterator over apps in given queues
+      // Collect list of lists to avoid copying all apps
+      final List<List<ApplicationAttemptId>> queueAppLists =
+          new ArrayList<List<ApplicationAttemptId>>();
+      for (String queue : queues) {
+        List<ApplicationAttemptId> appsInQueue = scheduler.getAppsInQueue(queue);
+        if (appsInQueue != null && !appsInQueue.isEmpty()) {
+          queueAppLists.add(appsInQueue);
+        }
       }
+      appsIter = new Iterator<RMApp>() {
+        Iterator<List<ApplicationAttemptId>> appListIter = queueAppLists.iterator();
+        Iterator<ApplicationAttemptId> schedAppsIter;
+
+        @Override
+        public boolean hasNext() {
+          // Because queueAppLists has no empty lists, hasNext is whether the
+          // current list hasNext or whether there are any remaining lists
+          return (schedAppsIter != null && schedAppsIter.hasNext())
+              || appListIter.hasNext();
+        }
+        @Override
+        public RMApp next() {
+          if (schedAppsIter == null || !schedAppsIter.hasNext()) {
+            schedAppsIter = appListIter.next().iterator();
+          }
+          return apps.get(schedAppsIter.next().getApplicationId());
+        }
+        @Override
+        public void remove() {
+          throw new UnsupportedOperationException("Remove not supported");
+        }
+      };
+    } else {
+      appsIter = apps.values().iterator();
+    }
+    
+    List<ApplicationReport> reports = new ArrayList<ApplicationReport>();
+    while (appsIter.hasNext() && reports.size() < limit) {
+      RMApp application = appsIter.next();
       if (applicationTypes != null && !applicationTypes.isEmpty()) {
         String appTypeToMatch = caseSensitive
             ? application.getApplicationType()
@@ -458,11 +500,6 @@ public class ClientRMService extends AbstractService implements
         continue;
       }
 
-      if (queues != null && !queues.isEmpty() &&
-          !queues.contains(application.getQueue())) {
-        continue;
-      }
-
       if (start != null && !start.containsLong(application.getStartTime())) {
         continue;
       }
@@ -515,13 +552,12 @@ public class ClientRMService extends AbstractService implements
             request.getRecursive());
       List<ApplicationReport> appReports = EMPTY_APPS_REPORT;
       if (request.getIncludeApplications()) {
-        Collection<RMApp> apps = this.rmContext.getRMApps().values();
-        appReports = new ArrayList<ApplicationReport>(
-            apps.size());
-        for (RMApp app : apps) {
-          if (app.getQueue().equals(queueInfo.getQueueName())) {
-            appReports.add(app.createAndGetApplicationReport(null, true));
-          }
+        List<ApplicationAttemptId> apps =
+            scheduler.getAppsInQueue(request.getQueueName());
+        appReports = new ArrayList<ApplicationReport>(apps.size());
+        for (ApplicationAttemptId app : apps) {
+          RMApp rmApp = rmContext.getRMApps().get(app.getApplicationId());
+          appReports.add(rmApp.createAndGetApplicationReport(null, true));
         }
       }
       queueInfo.setApplications(appReports);

+ 35 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java

@@ -65,7 +65,9 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
 
   private static final Log LOG = LogFactory.getLog(RMAppManager.class);
 
-  private int completedAppsMax = YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS;
+  private int maxCompletedAppsInMemory;
+  private int maxCompletedAppsInStateStore;
+  protected int completedAppsInStateStore = 0;
   private LinkedList<ApplicationId> completedApps = new LinkedList<ApplicationId>();
 
   private final RMContext rmContext;
@@ -82,9 +84,16 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     this.masterService = masterService;
     this.applicationACLsManager = applicationACLsManager;
     this.conf = conf;
-    setCompletedAppsMax(conf.getInt(
+    this.maxCompletedAppsInMemory = conf.getInt(
         YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS,
-        YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS));
+        YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS);
+    this.maxCompletedAppsInStateStore =
+        conf.getInt(
+          YarnConfiguration.RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS,
+          YarnConfiguration.DEFAULT_RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS);
+    if (this.maxCompletedAppsInStateStore > this.maxCompletedAppsInMemory) {
+      this.maxCompletedAppsInStateStore = this.maxCompletedAppsInMemory;
+    }
   }
 
   /**
@@ -173,10 +182,6 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     ApplicationSummary.logAppSummary(rmContext.getRMApps().get(appId));
   }
 
-  protected synchronized void setCompletedAppsMax(int max) {
-    this.completedAppsMax = max;
-  }
-
   protected synchronized int getCompletedAppsListSize() {
     return this.completedApps.size(); 
   }
@@ -190,7 +195,8 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
         rmContext.getDelegationTokenRenewer().applicationFinished(applicationId);
       }
       
-      completedApps.add(applicationId);  
+      completedApps.add(applicationId);
+      completedAppsInStateStore++;
       writeAuditLog(applicationId);
     }
   }
@@ -229,15 +235,31 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
    * check to see if hit the limit for max # completed apps kept
    */
   protected synchronized void checkAppNumCompletedLimit() {
-    while (completedApps.size() > this.completedAppsMax) {
-      ApplicationId removeId = completedApps.remove();  
-      LOG.info("Application should be expired, max # apps"
-          + " met. Removing app: " + removeId); 
+    // check apps kept in state store.
+    while (completedAppsInStateStore > this.maxCompletedAppsInStateStore) {
+      ApplicationId removeId =
+          completedApps.get(completedApps.size() - completedAppsInStateStore);
+      RMApp removeApp = rmContext.getRMApps().get(removeId);
+      LOG.info("Max number of completed apps kept in state store met:"
+          + " maxCompletedAppsInStateStore = " + maxCompletedAppsInStateStore
+          + ", removing app " + removeApp.getApplicationId()
+          + " from state store.");
+      rmContext.getStateStore().removeApplication(removeApp);
+      completedAppsInStateStore--;
+    }
+
+    // check apps kept in memorty.
+    while (completedApps.size() > this.maxCompletedAppsInMemory) {
+      ApplicationId removeId = completedApps.remove();
+      LOG.info("Application should be expired, max number of completed apps"
+          + " kept in memory met: maxCompletedAppsInMemory = "
+          + this.maxCompletedAppsInMemory + ", removing app " + removeId
+          + " from memory: ");
       rmContext.getRMApps().remove(removeId);
       this.applicationACLsManager.removeApplication(removeId);
     }
   }
-  
+
   @SuppressWarnings("unchecked")
   protected void submitApplication(
       ApplicationSubmissionContext submissionContext, long submitTime,
@@ -380,8 +402,6 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     Map<ApplicationId, ApplicationState> appStates = state.getApplicationState();
     LOG.info("Recovering " + appStates.size() + " applications");
     for (ApplicationState appState : appStates.values()) {
-      LOG.info("Recovering application " + appState.getAppId());
-      
       submitApplication(appState.getApplicationSubmissionContext(),
         appState.getSubmitTime(), appState.getUser(), true, state);
     }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java

@@ -362,7 +362,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
         // the Exception from stateStore.init() needs to be handled for
         // HA and we need to give up master status if we got fenced
         LOG.error("Failed to init state store", e);
-        ExitUtil.terminate(1, e);
+        throw e;
       }
       rmContext.setStateStore(rmStore);
 
@@ -470,7 +470,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
           // the Exception from loadState() needs to be handled for
           // HA and we need to give up master status if we got fenced
           LOG.error("Failed to load/recover state", e);
-          ExitUtil.terminate(1, e);
+          throw e;
         }
       }
 

+ 9 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java

@@ -167,7 +167,9 @@ public class FileSystemRMStateStore extends RMStateStore {
               readFile(childNodeStatus.getPath(), childNodeStatus.getLen());
           if (childNodeName.startsWith(ApplicationId.appIdStrPrefix)) {
             // application
-            LOG.info("Loading application from node: " + childNodeName);
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Loading application from node: " + childNodeName);
+            }
             ApplicationId appId = ConverterUtils.toApplicationId(childNodeName);
             ApplicationStateDataPBImpl appStateData =
                 new ApplicationStateDataPBImpl(
@@ -185,7 +187,10 @@ public class FileSystemRMStateStore extends RMStateStore {
           } else if (childNodeName
             .startsWith(ApplicationAttemptId.appAttemptIdStrPrefix)) {
             // attempt
-            LOG.info("Loading application attempt from node: " + childNodeName);
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Loading application attempt from node: "
+                  + childNodeName);
+            }
             ApplicationAttemptId attemptId =
                 ConverterUtils.toApplicationAttemptId(childNodeName);
             ApplicationAttemptStateDataPBImpl attemptStateData =
@@ -225,6 +230,7 @@ public class FileSystemRMStateStore extends RMStateStore {
         assert appState != null;
         appState.attempts.put(attemptState.getAttemptId(), attemptState);
       }
+      LOG.info("Done Loading applications from FS state store");
     } catch (Exception e) {
       LOG.error("Failed to load state.", e);
       throw e;
@@ -362,7 +368,7 @@ public class FileSystemRMStateStore extends RMStateStore {
   }
 
   @Override
-  public synchronized void removeApplicationState(ApplicationState appState)
+  public synchronized void removeApplicationStateInternal(ApplicationState appState)
       throws Exception {
     String appId = appState.getAppId().toString();
     Path nodeRemovePath = getAppDir(rmAppRoot, appId);

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java

@@ -171,8 +171,8 @@ public class MemoryRMStateStore extends RMStateStore {
   }
 
   @Override
-  public synchronized void removeApplicationState(ApplicationState appState) 
-                                                            throws Exception {
+  public synchronized void removeApplicationStateInternal(
+      ApplicationState appState) throws Exception {
     ApplicationId appId = appState.getAppId();
     ApplicationState removed = state.appState.remove(appId);
     if (removed == null) {

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/NullRMStateStore.java

@@ -63,7 +63,7 @@ public class NullRMStateStore extends RMStateStore {
   }
 
   @Override
-  protected void removeApplicationState(ApplicationState appState)
+  protected void removeApplicationStateInternal(ApplicationState appState)
       throws Exception {
     // Do nothing
   }

+ 4 - 25
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java

@@ -53,7 +53,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.Ap
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppNewSavedEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRemovedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppUpdateSavedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -519,6 +518,7 @@ public abstract class RMStateStore extends AbstractService {
    * This does not block the dispatcher threads
    * There is no notification of completion for this operation.
    */
+  @SuppressWarnings("unchecked")
   public synchronized void removeApplication(RMApp app) {
     ApplicationState appState = new ApplicationState(
             app.getSubmitTime(), app.getStartTime(),
@@ -532,14 +532,6 @@ public abstract class RMStateStore extends AbstractService {
       appState.attempts.put(attemptState.getAttemptId(), attemptState);
     }
     
-    removeApplication(appState);
-  }
-  
-  @SuppressWarnings("unchecked")
-  /**
-   * Non-Blocking API
-   */
-  public synchronized void removeApplication(ApplicationState appState) {
     dispatcher.getEventHandler().handle(new RMStateStoreRemoveAppEvent(appState));
   }
 
@@ -548,8 +540,8 @@ public abstract class RMStateStore extends AbstractService {
    * Derived classes must implement this method to remove the state of an 
    * application and its attempts
    */
-  protected abstract void removeApplicationState(ApplicationState appState) 
-                                                             throws Exception;
+  protected abstract void removeApplicationStateInternal(
+      ApplicationState appState) throws Exception;
 
   // TODO: This should eventually become cluster-Id + "AM_RM_TOKEN_SERVICE". See
   // YARN-986 
@@ -666,11 +658,9 @@ public abstract class RMStateStore extends AbstractService {
       ApplicationState appState =
           ((RMStateStoreRemoveAppEvent) event).getAppState();
       ApplicationId appId = appState.getAppId();
-      Exception removedException = null;
       LOG.info("Removing info for app: " + appId);
       try {
-        removeApplicationState(appState);
-        notifyDoneRemovingApplcation(appId, removedException);
+        removeApplicationStateInternal(appState);
       } catch (Exception e) {
         LOG.error("Error removing app: " + appId, e);
         notifyStoreOperationFailed(e);
@@ -738,17 +728,6 @@ public abstract class RMStateStore extends AbstractService {
       new RMAppAttemptUpdateSavedEvent(attemptId, updatedException));
   }
 
-  @SuppressWarnings("unchecked")
-  /**
-   * This is to notify RMApp that this application has been removed from
-   * RMStateStore
-   */
-  private void notifyDoneRemovingApplcation(ApplicationId appId,
-      Exception removedException) {
-    rmDispatcher.getEventHandler().handle(
-      new RMAppRemovedEvent(appId, removedException));
-  }
-
   /**
    * EventHandler implementation which forward events to the FSRMStateStore
    * This hides the EventHandle methods of the store from its public interface 

+ 14 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java

@@ -392,7 +392,9 @@ public class ZKRMStateStore extends RMStateStore {
       byte[] childData = getDataWithRetries(childNodePath, true);
       if (childNodeName.startsWith(ApplicationId.appIdStrPrefix)) {
         // application
-        LOG.info("Loading application from znode: " + childNodeName);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Loading application from znode: " + childNodeName);
+        }
         ApplicationId appId = ConverterUtils.toApplicationId(childNodeName);
         ApplicationStateDataPBImpl appStateData =
             new ApplicationStateDataPBImpl(
@@ -412,7 +414,9 @@ public class ZKRMStateStore extends RMStateStore {
       } else if (childNodeName
           .startsWith(ApplicationAttemptId.appAttemptIdStrPrefix)) {
         // attempt
-        LOG.info("Loading application attempt from znode: " + childNodeName);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Loading application attempt from znode: " + childNodeName);
+        }
         ApplicationAttemptId attemptId =
             ConverterUtils.toApplicationAttemptId(childNodeName);
         ApplicationAttemptStateDataPBImpl attemptStateData =
@@ -456,10 +460,10 @@ public class ZKRMStateStore extends RMStateStore {
         LOG.info("Application node not found for attempt: "
             + attemptState.getAttemptId());
         deleteWithRetries(
-            getNodePath(rmAppRoot, attemptState.getAttemptId().toString()),
-            0);
+            getNodePath(rmAppRoot, attemptState.getAttemptId().toString()), -1);
       }
     }
+    LOG.info("Done Loading applications from ZK state store");
   }
 
   @Override
@@ -517,16 +521,16 @@ public class ZKRMStateStore extends RMStateStore {
   }
 
   @Override
-  public synchronized void removeApplicationState(ApplicationState appState)
+  public synchronized void removeApplicationStateInternal(ApplicationState appState)
       throws Exception {
     String appId = appState.getAppId().toString();
     String nodeRemovePath = getNodePath(rmAppRoot, appId);
     ArrayList<Op> opList = new ArrayList<Op>();
-    opList.add(Op.delete(nodeRemovePath, 0));
+    opList.add(Op.delete(nodeRemovePath, -1));
 
     for (ApplicationAttemptId attemptId : appState.attempts.keySet()) {
       String attemptRemovePath = getNodePath(rmAppRoot, attemptId.toString());
-      opList.add(Op.delete(attemptRemovePath, 0));
+      opList.add(Op.delete(attemptRemovePath, -1));
     }
     if (LOG.isDebugEnabled()) {
       LOG.debug("Removing info for app: " + appId + " at: " + nodeRemovePath
@@ -569,7 +573,7 @@ public class ZKRMStateStore extends RMStateStore {
     }
 
     if (dtSequenceNumberPath != null) {
-      opList.add(Op.delete(dtSequenceNumberPath, 0));
+      opList.add(Op.delete(dtSequenceNumberPath, -1));
     }
     opList.add(Op.create(latestSequenceNumberPath, null, zkAcl,
         CreateMode.PERSISTENT));
@@ -587,7 +591,7 @@ public class ZKRMStateStore extends RMStateStore {
       LOG.debug("Removing RMDelegationToken_"
           + rmDTIdentifier.getSequenceNumber());
     }
-    deleteWithRetries(nodeRemovePath, 0);
+    deleteWithRetries(nodeRemovePath, -1);
   }
 
   @Override
@@ -619,7 +623,7 @@ public class ZKRMStateStore extends RMStateStore {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Removing RMDelegationKey_" + delegationKey.getKeyId());
     }
-    deleteWithRetries(nodeRemovePath, 0);
+    deleteWithRetries(nodeRemovePath, -1);
   }
 
   // ZK related code

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

@@ -660,32 +660,34 @@ public class RMAppImpl implements RMApp, Recoverable {
   @SuppressWarnings("unchecked")
   private static final class RMAppRecoveredTransition implements
       MultipleArcTransition<RMAppImpl, RMAppEvent, RMAppState> {
-    
+
     @Override
     public RMAppState transition(RMAppImpl app, RMAppEvent event) {
+      /*
+       * If last attempt recovered final state is null .. it means attempt was
+       * started but AM container may or may not have started / finished.
+       * Therefore we should wait for it to finish.
+       */
+      for (RMAppAttempt attempt : app.getAppAttempts().values()) {
+        app.dispatcher.getEventHandler().handle(
+          new RMAppAttemptEvent(attempt.getAppAttemptId(),
+            RMAppAttemptEventType.RECOVER));
+      }
+
+      // The app has completed.
+      if (app.recoveredFinalState != null) {
+        FINAL_TRANSITION.transition(app, event);
+        return app.recoveredFinalState;
+      }
 
+      // No existent attempts means the attempt associated with this app was not
+      // started or started but not yet saved。
       if (app.attempts.isEmpty()) {
-        // Saved application was not running any attempts.
         app.createNewAttempt(true);
-        return RMAppState.SUBMITTED;        
-      } else {
-        /*
-         * If last attempt recovered final state is null .. it means attempt
-         * was started but AM container may or may not have started / finished.
-         * Therefore we should wait for it to finish.
-         */
-        for (RMAppAttempt attempt : app.getAppAttempts().values()) {
-          app.dispatcher.getEventHandler().handle(
-              new RMAppAttemptEvent(attempt.getAppAttemptId(),
-                  RMAppAttemptEventType.RECOVER));
-        }        
-        if (app.recoveredFinalState != null) {
-          FINAL_TRANSITION.transition(app, event);
-          return app.recoveredFinalState;
-        } else {
-          return RMAppState.RUNNING;
-        }
+        return RMAppState.SUBMITTED;
       }
+
+      return RMAppState.RUNNING;
     }
   }
 

+ 0 - 36
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppRemovedEvent.java

@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
-
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-
-public class RMAppRemovedEvent extends RMAppEvent {
-
-  private final Exception removedException;
-
-  public RMAppRemovedEvent(ApplicationId appId, Exception removedException) {
-    super(appId, RMAppEventType.APP_REMOVED);
-    this.removedException = removedException;
-  }
-
-  public Exception getRemovedException() {
-    return removedException;
-  }
-
-}

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

@@ -76,14 +76,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEve
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFinishedAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRejectedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStatusupdateEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUpdateSavedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
@@ -675,9 +674,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     ApplicationAttemptState attemptState =
         appState.getAttempt(getAppAttemptId());
     assert attemptState != null;
-    LOG.info("Recovered attempt: AppId: "
-        + getAppAttemptId().getApplicationId() + " AttemptId: "
-        + getAppAttemptId() + " MasterContainer: " + masterContainer);
+    LOG.info("Recovering attempt: " + getAppAttemptId() + " with final state: "
+        + attemptState.getState());
     diagnostics.append("Attempt recovered after RM restart");
     diagnostics.append(attemptState.getDiagnostics());
     setMasterContainer(attemptState.getMasterContainer());
@@ -856,8 +854,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     @Override
     public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
         RMAppAttemptEvent event) {
-      LOG.info("Recovering attempt :  recoverdFinalState :"
-          + appAttempt.recoveredFinalState);
       if (appAttempt.recoveredFinalState != null) {
         appAttempt.progress = 1.0f;
         RMApp rmApp =appAttempt.rmContext.getRMApps().get(

+ 10 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 import java.io.IOException;
+import java.util.Collection;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
@@ -160,4 +161,13 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
    */
   boolean checkAccess(UserGroupInformation callerUGI,
       QueueACL acl, String queueName);
+  
+  /**
+   * Gets the apps under a given queue
+   * @param queueName the name of the queue.
+   * @return a collection of app attempt ids in the given queue.
+   */
+  @LimitedPrivate("yarn")
+  @Stable
+  public List<ApplicationAttemptId> getAppsInQueue(String queueName);
 }

+ 9 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java

@@ -19,12 +19,14 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import java.io.IOException;
+import java.util.Collection;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -33,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 
@@ -228,4 +231,10 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
    */
   public void recoverContainer(Resource clusterResource, FiCaSchedulerApp application, 
       Container container);
+  
+  /**
+   * Adds all applications in the queue and its subqueues to the given collection.
+   * @param apps the collection to add the applications to
+   */
+  public void collectSchedulerApplications(Collection<ApplicationAttemptId> apps);
 }

+ 11 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java

@@ -66,6 +66,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
@@ -941,4 +942,14 @@ public class CapacityScheduler
     return queue.hasAccess(acl, callerUGI);
   }
 
+  @Override
+  public List<ApplicationAttemptId> getAppsInQueue(String queueName) {
+    CSQueue queue = queues.get(queueName);
+    if (queue == null) {
+      return null;
+    }
+    List<ApplicationAttemptId> apps = new ArrayList<ApplicationAttemptId>();
+    queue.collectSchedulerApplications(apps);
+    return apps;
+  }
 }

+ 10 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -58,6 +59,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -1622,4 +1624,12 @@ public class LeafQueue implements CSQueue {
     return ret;
   }
 
+  @Override
+  public void collectSchedulerApplications(
+      Collection<ApplicationAttemptId> apps) {
+    for (FiCaSchedulerApp app : activeApplications) {
+      apps.add(app.getApplicationAttemptId());
+    }
+  }
+
 }

+ 10 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -50,6 +51,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@@ -764,4 +766,12 @@ public class ParentQueue implements CSQueue {
       parent.recoverContainer(clusterResource, application, container);
     }
   }
+
+  @Override
+  public void collectSchedulerApplications(
+      Collection<ApplicationAttemptId> apps) {
+    for (CSQueue queue : childQueues) {
+      queue.collectSchedulerApplications(apps);
+    }
+  }
 }

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

@@ -29,11 +29,13 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 
 @Private
 @Unstable
@@ -105,6 +107,17 @@ public class FSLeafQueue extends FSQueue {
   public List<AppSchedulable> getNonRunnableAppSchedulables() {
     return nonRunnableAppScheds;
   }
+  
+  @Override
+  public void collectSchedulerApplications(
+      Collection<ApplicationAttemptId> apps) {
+    for (AppSchedulable appSched : runnableAppScheds) {
+      apps.add(appSched.getApp().getApplicationAttemptId());
+    }
+    for (AppSchedulable appSched : nonRunnableAppScheds) {
+      apps.add(appSched.getApp().getApplicationAttemptId());
+    }
+  }
 
   @Override
   public void setPolicy(SchedulingPolicy policy)

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

@@ -28,10 +28,12 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.util.resource.Resources;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 
 @Private
 @Unstable
@@ -184,4 +186,12 @@ public class FSParentQueue extends FSQueue {
   public int getNumRunnableApps() {
     return runnableApps;
   }
+
+  @Override
+  public void collectSchedulerApplications(
+      Collection<ApplicationAttemptId> apps) {
+    for (FSQueue childQueue : childQueues) {
+      childQueue.collectSchedulerApplications(apps);
+    }
+  }
 }

+ 9 - 1
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

@@ -24,6 +24,7 @@ import java.util.Collection;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
@@ -158,7 +159,14 @@ public abstract class FSQueue extends Schedulable implements Queue {
    * Gets the children of this queue, if any.
    */
   public abstract Collection<FSQueue> getChildQueues();
-
+  
+  /**
+   * Adds all applications in the queue and its subqueues to the given collection.
+   * @param apps the collection to add the applications to
+   */
+  public abstract void collectSchedulerApplications(
+      Collection<ApplicationAttemptId> apps);
+  
   /**
    * Return the number of apps for which containers can be allocated.
    * Includes apps in subqueues.

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

@@ -72,6 +72,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
@@ -1267,4 +1268,15 @@ public class FairScheduler implements ResourceScheduler {
     }
   }
 
+  @Override
+  public List<ApplicationAttemptId> getAppsInQueue(String queueName) {
+    FSQueue queue = queueMgr.getQueue(queueName);
+    if (queue == null) {
+      return null;
+    }
+    List<ApplicationAttemptId> apps = new ArrayList<ApplicationAttemptId>();
+    queue.collectSchedulerApplications(apps);
+    return apps;
+  }
+
 }

+ 15 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -850,5 +851,19 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
       QueueACL acl, String queueName) {
     return DEFAULT_QUEUE.hasAccess(acl, callerUGI);
   }
+  
+  @Override
+  public synchronized List<ApplicationAttemptId> getAppsInQueue(String queueName) {
+    if (queueName.equals(DEFAULT_QUEUE.getQueueName())) {
+      List<ApplicationAttemptId> apps = new ArrayList<ApplicationAttemptId>(
+          applications.size());
+      for (FiCaSchedulerApp app : applications.values()) {
+        apps.add(app.getApplicationAttemptId());
+      }
+      return apps;
+    } else {
+      return null;
+    }
+  }
 
 }

+ 91 - 20
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java

@@ -19,8 +19,12 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 
+import static org.mockito.Matchers.isA;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.times;
 
 import java.util.HashMap;
 import java.util.List;
@@ -43,6 +47,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.MockRMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
@@ -99,7 +104,7 @@ public class TestAppManager{
         rmDispatcher);
     AMLivelinessMonitor amFinishingMonitor = new AMLivelinessMonitor(
         rmDispatcher);
-    return new RMContextImpl(rmDispatcher,
+    RMContext context = new RMContextImpl(rmDispatcher,
         containerAllocationExpirer, amLivelinessMonitor, amFinishingMonitor,
         null, null, null, null, null) {
       @Override
@@ -107,6 +112,8 @@ public class TestAppManager{
         return map;
       }
     };
+    ((RMContextImpl)context).setStateStore(mock(RMStateStore.class));
+    return context;
   }
 
   public class TestAppManagerDispatcher implements
@@ -142,7 +149,6 @@ public class TestAppManager{
 
     public TestRMAppManager(RMContext context, Configuration conf) {
       super(context, null, null, new ApplicationACLsManager(conf), conf);
-      setCompletedAppsMax(YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS);
     }
 
     public TestRMAppManager(RMContext context,
@@ -150,7 +156,6 @@ public class TestAppManager{
         YarnScheduler scheduler, ApplicationMasterService masterService,
         ApplicationACLsManager applicationACLsManager, Configuration conf) {
       super(context, scheduler, masterService, applicationACLsManager, conf);
-      setCompletedAppsMax(YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS);
     }
 
     public void checkAppNumCompletedLimit() {
@@ -164,9 +169,8 @@ public class TestAppManager{
     public int getCompletedAppsListSize() {
       return super.getCompletedAppsListSize();
     }
-
-    public void setCompletedAppsMax(int max) {
-      super.setCompletedAppsMax(max);
+    public int getCompletedAppsInStateStore() {
+      return this.completedAppsInStateStore;
     }
     public void submitApplication(
         ApplicationSubmissionContext submissionContext, String user)
@@ -227,9 +231,9 @@ public class TestAppManager{
     // Create such that none of the applications will retire since
     // haven't hit max #
     RMContext rmContext = mockRMContext(10, now - 10);
-    TestRMAppManager appMonitor = new TestRMAppManager(rmContext, new Configuration());
-
-    appMonitor.setCompletedAppsMax(10);
+    Configuration conf = new YarnConfiguration();
+    conf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 10);
+    TestRMAppManager appMonitor = new TestRMAppManager(rmContext,conf);
 
     Assert.assertEquals("Number of apps incorrect before checkAppTimeLimit",
         10, rmContext.getRMApps().size());
@@ -243,6 +247,8 @@ public class TestAppManager{
         rmContext.getRMApps().size());
     Assert.assertEquals("Number of completed apps incorrect after check", 10,
         appMonitor.getCompletedAppsListSize());
+    verify(rmContext.getStateStore(), never()).removeApplication(
+      isA(RMApp.class));
   }
 
   @Test
@@ -250,9 +256,10 @@ public class TestAppManager{
     long now = System.currentTimeMillis();
 
     RMContext rmContext = mockRMContext(10, now - 20000);
-    TestRMAppManager appMonitor = new TestRMAppManager(rmContext, new Configuration());
-
-    appMonitor.setCompletedAppsMax(3);
+    Configuration conf = new YarnConfiguration();
+    conf.setInt(YarnConfiguration.RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS, 3); 
+    conf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 3);
+    TestRMAppManager appMonitor = new TestRMAppManager(rmContext, conf);
 
     Assert.assertEquals("Number of apps incorrect before", 10, rmContext
         .getRMApps().size());
@@ -266,6 +273,8 @@ public class TestAppManager{
         rmContext.getRMApps().size());
     Assert.assertEquals("Number of completed apps incorrect after check", 3,
         appMonitor.getCompletedAppsListSize());
+    verify(rmContext.getStateStore(), times(7)).removeApplication(
+      isA(RMApp.class));
   }
 
   @Test
@@ -274,14 +283,17 @@ public class TestAppManager{
 
     // these parameters don't matter, override applications below
     RMContext rmContext = mockRMContext(10, now - 20000);
-    TestRMAppManager appMonitor = new TestRMAppManager(rmContext, new Configuration());
+    Configuration conf = new YarnConfiguration();
+    conf.setInt(YarnConfiguration.RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS, 2);
+    conf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 2);
 
-    appMonitor.setCompletedAppsMax(2);
+    TestRMAppManager appMonitor = new TestRMAppManager(rmContext, conf);
 
     // clear out applications map
     rmContext.getRMApps().clear();
     Assert.assertEquals("map isn't empty", 0, rmContext.getRMApps().size());
 
+    // 6 applications are in final state, 4 are not in final state.
     // / set with various finished states
     RMApp app = new MockRMApp(0, now - 20000, RMAppState.KILLED);
     rmContext.getRMApps().put(app.getApplicationId(), app);
@@ -318,7 +330,9 @@ public class TestAppManager{
         rmContext.getRMApps().size());
     Assert.assertEquals("Number of completed apps incorrect after check", 2,
         appMonitor.getCompletedAppsListSize());
-
+    // 6 applications in final state, 4 of them are removed
+    verify(rmContext.getStateStore(), times(4)).removeApplication(
+      isA(RMApp.class));
   }
 
   @Test
@@ -342,14 +356,13 @@ public class TestAppManager{
     long now = System.currentTimeMillis();
 
     RMContext rmContext = mockRMContext(10, now - 20000);
-    TestRMAppManager appMonitor = new TestRMAppManager(rmContext, new Configuration());
-
+    Configuration conf = new YarnConfiguration();
+    conf.setInt(YarnConfiguration.RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS, 0);
+    conf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 0);
+    TestRMAppManager appMonitor = new TestRMAppManager(rmContext, conf);
     Assert.assertEquals("Number of apps incorrect before", 10, rmContext
         .getRMApps().size());
 
-    // test with 0
-    appMonitor.setCompletedAppsMax(0);
-
     addToCompletedApps(appMonitor, rmContext);
     Assert.assertEquals("Number of completed apps incorrect", 10,
         appMonitor.getCompletedAppsListSize());
@@ -360,6 +373,64 @@ public class TestAppManager{
         rmContext.getRMApps().size());
     Assert.assertEquals("Number of completed apps incorrect after check", 0,
         appMonitor.getCompletedAppsListSize());
+    verify(rmContext.getStateStore(), times(10)).removeApplication(
+      isA(RMApp.class));
+  }
+
+  @Test
+  public void testStateStoreAppLimitLessThanMemoryAppLimit() {
+    long now = System.currentTimeMillis();
+    RMContext rmContext = mockRMContext(10, now - 20000);
+    Configuration conf = new YarnConfiguration();
+    int maxAppsInMemory = 8;
+    int maxAppsInStateStore = 4;
+    conf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, maxAppsInMemory);
+    conf.setInt(YarnConfiguration.RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS,
+      maxAppsInStateStore);
+    TestRMAppManager appMonitor = new TestRMAppManager(rmContext, conf);
+
+    addToCompletedApps(appMonitor, rmContext);
+    Assert.assertEquals("Number of completed apps incorrect", 10,
+        appMonitor.getCompletedAppsListSize());
+    appMonitor.checkAppNumCompletedLimit();
+
+    Assert.assertEquals("Number of apps incorrect after # completed check",
+      maxAppsInMemory, rmContext.getRMApps().size());
+    Assert.assertEquals("Number of completed apps incorrect after check",
+      maxAppsInMemory, appMonitor.getCompletedAppsListSize());
+
+    int numRemoveAppsFromStateStore = 10 - maxAppsInStateStore;
+    verify(rmContext.getStateStore(), times(numRemoveAppsFromStateStore))
+      .removeApplication(isA(RMApp.class));
+    Assert.assertEquals(maxAppsInStateStore,
+      appMonitor.getCompletedAppsInStateStore());
+  }
+
+  @Test
+  public void testStateStoreAppLimitLargerThanMemoryAppLimit() {
+    long now = System.currentTimeMillis();
+    RMContext rmContext = mockRMContext(10, now - 20000);
+    Configuration conf = new YarnConfiguration();
+    int maxAppsInMemory = 8;
+    conf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, maxAppsInMemory);
+    // larger than maxCompletedAppsInMemory, reset to RM_MAX_COMPLETED_APPLICATIONS.
+    conf.setInt(YarnConfiguration.RM_STATE_STORE_MAX_COMPLETED_APPLICATIONS, 1000);
+    TestRMAppManager appMonitor = new TestRMAppManager(rmContext, conf);
+
+    addToCompletedApps(appMonitor, rmContext);
+    Assert.assertEquals("Number of completed apps incorrect", 10,
+        appMonitor.getCompletedAppsListSize());
+    appMonitor.checkAppNumCompletedLimit();
+
+    int numRemoveApps = 10 - maxAppsInMemory;
+    Assert.assertEquals("Number of apps incorrect after # completed check",
+      maxAppsInMemory, rmContext.getRMApps().size());
+    Assert.assertEquals("Number of completed apps incorrect after check",
+      maxAppsInMemory, appMonitor.getCompletedAppsListSize());
+    verify(rmContext.getStateStore(), times(numRemoveApps)).removeApplication(
+      isA(RMApp.class));
+    Assert.assertEquals(maxAppsInMemory,
+      appMonitor.getCompletedAppsInStateStore());
   }
 
   protected void setupDispatcher(RMContext rmContext, Configuration conf) {

+ 27 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java

@@ -30,9 +30,12 @@ import static org.mockito.Mockito.spy;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashSet;
 import java.util.EnumSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.ConcurrentHashMap;
@@ -106,6 +109,9 @@ public class TestClientRMService {
 
   private static RMDelegationTokenSecretManager dtsm;
   
+  private final static String QUEUE_1 = "Q-1";
+  private final static String QUEUE_2 = "Q-2";
+  
   @BeforeClass
   public static void setupSecretManager() throws IOException {
     RMContext rmContext = mock(RMContext.class);
@@ -438,7 +444,7 @@ public class TestClientRMService {
             mockAclsManager, mockQueueACLsManager, null);
 
     // Initialize appnames and queues
-    String[] queues = {"Q-1", "Q-2"};
+    String[] queues = {QUEUE_1, QUEUE_2};
     String[] appNames =
         {MockApps.newAppName(), MockApps.newAppName(), MockApps.newAppName()};
     ApplicationId[] appIds =
@@ -596,6 +602,8 @@ public class TestClientRMService {
     ConcurrentHashMap<ApplicationId, RMApp> apps = getRMApps(rmContext,
         yarnScheduler);
     when(rmContext.getRMApps()).thenReturn(apps);
+    when(yarnScheduler.getAppsInQueue(eq("testqueue"))).thenReturn(
+        getSchedulerApps(apps));
   }
 
   private ConcurrentHashMap<ApplicationId, RMApp> getRMApps(
@@ -614,10 +622,23 @@ public class TestClientRMService {
         config, "testqueue"));
     return apps;
   }
+  
+  private List<ApplicationAttemptId> getSchedulerApps(
+      Map<ApplicationId, RMApp> apps) {
+    List<ApplicationAttemptId> schedApps = new ArrayList<ApplicationAttemptId>();
+    // Return app IDs for the apps in testqueue (as defined in getRMApps)
+    schedApps.add(ApplicationAttemptId.newInstance(getApplicationId(1), 0));
+    schedApps.add(ApplicationAttemptId.newInstance(getApplicationId(3), 0));
+    return schedApps;
+  }
 
-  private ApplicationId getApplicationId(int id) {
+  private static ApplicationId getApplicationId(int id) {
     return ApplicationId.newInstance(123456, id);
   }
+  
+  private static ApplicationAttemptId getApplicationAttemptId(int id) {
+    return ApplicationAttemptId.newInstance(getApplicationId(id), 1);
+  }
 
   private RMAppImpl getRMApp(RMContext rmContext, YarnScheduler yarnScheduler,
       ApplicationId applicationId3, YarnConfiguration config, String queueName) {
@@ -641,6 +662,10 @@ public class TestClientRMService {
     when(yarnScheduler.getMaximumResourceCapability()).thenReturn(
         Resources.createResource(
             YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB));
+    when(yarnScheduler.getAppsInQueue(QUEUE_1)).thenReturn(
+        Arrays.asList(getApplicationAttemptId(101), getApplicationAttemptId(102)));
+    when(yarnScheduler.getAppsInQueue(QUEUE_2)).thenReturn(
+        Arrays.asList(getApplicationAttemptId(103)));
     return yarnScheduler;
   }
 }

+ 123 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java

@@ -34,7 +34,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.LinkedBlockingQueue;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -46,7 +45,7 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
-import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.service.Service.STATE;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
@@ -68,9 +67,6 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.hadoop.yarn.event.Dispatcher;
-import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
@@ -80,6 +76,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
@@ -93,7 +90,6 @@ import org.apache.log4j.Logger;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.mortbay.log.Log;
 
 public class TestRMRestart {
 
@@ -106,7 +102,6 @@ public class TestRMRestart {
   public void setup() throws UnknownHostException {
     Logger rootLogger = LogManager.getRootLogger();
     rootLogger.setLevel(Level.DEBUG);
-    ExitUtil.disableSystemExit();
     conf = new YarnConfiguration();
     UserGroupInformation.setConfiguration(conf);
     conf.set(YarnConfiguration.RECOVERY_ENABLED, "true");
@@ -423,6 +418,8 @@ public class TestRMRestart {
         rm2.getRMContext().getRMApps().get(app0.getApplicationId());
     Assert.assertEquals(RMAppAttemptState.FAILED, recoveredApp
       .getAppAttempts().get(am0.getApplicationAttemptId()).getAppAttemptState());
+    rm1.stop();
+    rm2.stop();
   }
 
   @Test
@@ -629,6 +626,8 @@ public class TestRMRestart {
       .contains("Failing the application."));
     // failed diagnostics from attempt is lost because the diagnostics from
     // attempt is not yet available by the time app is saving the app state.
+    rm1.stop();
+    rm2.stop();
   }
 
   @Test
@@ -675,6 +674,48 @@ public class TestRMRestart {
     ApplicationReport appReport = verifyAppReportAfterRMRestart(app0, rm2);
     Assert.assertEquals(app0.getDiagnostics().toString(),
       appReport.getDiagnostics());
+    rm1.stop();
+    rm2.stop();
+  }
+
+  @Test
+  public void testRMRestartKilledAppWithNoAttempts() throws Exception {
+    MemoryRMStateStore memStore = new MemoryRMStateStore() {
+      @Override
+      public synchronized void storeApplicationAttemptStateInternal(
+          String attemptIdStr,
+          ApplicationAttemptStateDataPBImpl attemptStateData) throws Exception {
+        // ignore attempt saving request.
+      }
+
+      @Override
+      public synchronized void updateApplicationAttemptStateInternal(
+          String attemptIdStr,
+          ApplicationAttemptStateDataPBImpl attemptStateData) throws Exception {
+        // ignore attempt saving request.
+      }
+    };
+    memStore.init(conf);
+
+    // start RM
+    MockRM rm1 = new MockRM(conf, memStore);
+    rm1.start();
+    // create app
+    RMApp app0 =
+        rm1.submitApp(200, "name", "user",
+          new HashMap<ApplicationAccessType, String>(), false, "default", -1,
+          null, "MAPREDUCE", false);
+    // kill the app.
+    rm1.killApp(app0.getApplicationId());
+    rm1.waitForState(app0.getApplicationId(), RMAppState.KILLED);
+
+    // restart rm
+    MockRM rm2 = new MockRM(conf, memStore);
+    rm2.start();
+    RMApp loadedApp0 =
+        rm2.getRMContext().getRMApps().get(app0.getApplicationId());
+    rm2.waitForState(loadedApp0.getApplicationId(), RMAppState.KILLED);
+    Assert.assertTrue(loadedApp0.getAppAttempts().size() == 0);
   }
 
   @Test
@@ -724,6 +765,9 @@ public class TestRMRestart {
     Assert.assertEquals(FinalApplicationStatus.SUCCEEDED,
       appReport.getFinalApplicationStatus());
     Assert.assertEquals("trackingUrl", appReport.getOriginalTrackingUrl());
+
+    rm1.stop();
+    rm2.stop();
   }
 
   @Test
@@ -817,6 +861,9 @@ public class TestRMRestart {
     // check application summary is logged for the completed apps after RM restart.
     verify(rm2.getRMAppManager(), times(3)).logApplicationSummary(
       isA(ApplicationId.class));
+
+    rm1.stop();
+    rm2.stop();
   }
 
   private MockAM launchAM(RMApp app, MockRM rm, MockNM nm)
@@ -1378,6 +1425,75 @@ public class TestRMRestart {
     Assert.assertTrue(rmAppState.size() == NUM_APPS);
   }
 
+  @Test
+  public void testFinishedAppRemovalAfterRMRestart() throws Exception {
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    conf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 1);
+    memStore.init(conf);
+    RMState rmState = memStore.getState();
+
+    // start RM
+    MockRM rm1 = new MockRM(conf, memStore);
+    rm1.start();
+    MockNM nm1 =
+        new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
+    nm1.registerNode();
+
+    // create an app and finish the app.
+    RMApp app0 = rm1.submitApp(200);
+    MockAM am0 = launchAM(app0, rm1, nm1);
+    finishApplicationMaster(app0, rm1, nm1, am0);
+
+    MockRM rm2 = new MockRM(conf, memStore);
+    rm2.start();
+    nm1.setResourceTrackerService(rm2.getResourceTrackerService());
+    nm1 = rm2.registerNode("127.0.0.1:1234", 15120);
+
+    Map<ApplicationId, ApplicationState> rmAppState =
+        rmState.getApplicationState();
+
+    // app0 exits in both state store and rmContext
+    Assert.assertEquals(RMAppState.FINISHED,
+      rmAppState.get(app0.getApplicationId()).getState());
+    rm2.waitForState(app0.getApplicationId(), RMAppState.FINISHED);
+
+    // create one more app and finish the app.
+    RMApp app1 = rm2.submitApp(200);
+    MockAM am1 = launchAM(app1, rm2, nm1);
+    finishApplicationMaster(app1, rm2, nm1, am1);
+
+    // the first app0 get kicked out from both rmContext and state store
+    Assert.assertNull(rm2.getRMContext().getRMApps()
+      .get(app0.getApplicationId()));
+    Assert.assertNull(rmAppState.get(app0.getApplicationId()));
+
+    rm1.stop();
+    rm2.stop();
+  }
+
+  // This is to test RM does not get hang on shutdown.
+  @Test (timeout = 10000)
+  public void testRMShutdown() throws Exception {
+    MemoryRMStateStore memStore = new MemoryRMStateStore() {
+      @Override
+      public synchronized void checkVersion()
+          throws Exception {
+        throw new Exception("Invalid version.");
+      }
+    };
+    // start RM
+    memStore.init(conf);
+    MockRM rm1 = null;
+    try {
+      rm1 = new MockRM(conf, memStore);
+      rm1.start();
+      Assert.fail();
+    } catch (Exception e) {
+      Assert.assertTrue(e.getMessage().contains("Invalid version."));
+    }
+    Assert.assertTrue(rm1.getServiceState() == STATE.STOPPED);
+  }
+
   public static class TestSecurityMockRM extends MockRM {
 
     public TestSecurityMockRM(Configuration conf, RMStateStore store) {

+ 44 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java

@@ -26,6 +26,7 @@ import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -109,6 +110,7 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
     boolean isFinalStateValid() throws Exception;
     void writeVersion(RMStateVersion version) throws Exception;
     RMStateVersion getCurrentVersion() throws Exception;
+    boolean appExists(RMApp app) throws Exception;
   }
 
   void waitNotify(TestDispatcher dispatcher) {
@@ -128,7 +130,7 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
     dispatcher.notified = false;
   }
 
-  void storeApp(RMStateStore store, ApplicationId appId, long submitTime,
+  RMApp storeApp(RMStateStore store, ApplicationId appId, long submitTime,
       long startTime) throws Exception {
     ApplicationSubmissionContext context =
         new ApplicationSubmissionContextPBImpl();
@@ -141,6 +143,7 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
     when(mockApp.getApplicationSubmissionContext()).thenReturn(context);
     when(mockApp.getUser()).thenReturn("test");
     store.storeNewApplication(mockApp);
+    return mockApp;
   }
 
   ContainerId storeAttempt(RMStateStore store, ApplicationAttemptId attemptId,
@@ -370,6 +373,7 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
     Assert.assertEquals(keySet, secretManagerState.getMasterKeyState());
     Assert.assertEquals(sequenceNumber,
         secretManagerState.getDTSequenceNumber());
+    store.close();
   }
 
   private Token<AMRMTokenIdentifier> generateAMRMToken(
@@ -415,4 +419,43 @@ public class RMStateStoreTestBase extends ClientBaseWithFixes{
       Assert.assertTrue(t instanceof RMStateVersionIncompatibleException);
     }
   }
+
+  public void testAppDeletion(RMStateStoreHelper stateStoreHelper)
+      throws Exception {
+    RMStateStore store = stateStoreHelper.getRMStateStore();
+    store.setRMDispatcher(new TestDispatcher());
+    // create and store apps
+    ArrayList<RMApp> appList = new ArrayList<RMApp>();
+    int NUM_APPS = 5;
+    for (int i = 0; i < NUM_APPS; i++) {
+      ApplicationId appId = ApplicationId.newInstance(1383183338, i);
+      RMApp app = storeApp(store, appId, 123456789, 987654321);
+      appList.add(app);
+    }
+
+    Assert.assertEquals(NUM_APPS, appList.size());
+    for (RMApp app : appList) {
+      // wait for app to be stored.
+      while (true) {
+        if (stateStoreHelper.appExists(app)) {
+          break;
+        } else {
+          Thread.sleep(100);
+        }
+      }
+    }
+
+    for (RMApp app : appList) {
+      // remove the app
+      store.removeApplication(app);
+      // wait for app to be removed.
+      while (true) {
+        if (!stateStoreHelper.appExists(app)) {
+          break;
+        } else {
+          Thread.sleep(100);
+        }
+      }
+    }
+  }
 }

+ 20 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.recovery;
 
 import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import junit.framework.Assert;
@@ -38,6 +39,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.RMStateVersion;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.RMStateVersionPBImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.Test;
@@ -69,6 +71,13 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
       public RMStateVersion getCurrentVersion() {
         return CURRENT_VERSION_INFO;
       }
+
+      public Path getAppDir(String appId) {
+        Path rootDir = new Path(workingDirPathURI, ROOT_DIR_NAME);
+        Path appRootDir = new Path(rootDir, RM_APP_ROOT);
+        Path appDir = new Path(appRootDir, appId);
+        return appDir;
+      }
     }
 
     public TestFSRMStateStoreTester(MiniDFSCluster cluster) throws Exception {
@@ -109,9 +118,16 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
     public RMStateVersion getCurrentVersion() throws Exception {
       return store.getCurrentVersion();
     }
+
+    public boolean appExists(RMApp app) throws IOException {
+      FileSystem fs = cluster.getFileSystem();
+      Path nodePath =
+          store.getAppDir(app.getApplicationId().toString());
+      return fs.exists(nodePath);
+    }
   }
 
-  @Test
+  @Test(timeout = 60000)
   public void testFSRMStateStore() throws Exception {
     HdfsConfiguration conf = new HdfsConfiguration();
     MiniDFSCluster cluster =
@@ -126,11 +142,8 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
       String appAttemptIdStr3 = "appattempt_1352994193343_0001_000003";
       ApplicationAttemptId attemptId3 =
           ConverterUtils.toApplicationAttemptId(appAttemptIdStr3);
-      Path rootDir =
-          new Path(fileSystemRMStateStore.fsWorkingPath, "FSRMStateRoot");
-      Path appRootDir = new Path(rootDir, "RMAppRoot");
       Path appDir =
-          new Path(appRootDir, attemptId3.getApplicationId().toString());
+          fsTester.store.getAppDir(attemptId3.getApplicationId().toString());
       Path tempAppAttemptFile =
           new Path(appDir, attemptId3.toString() + ".tmp");
       fsOut = fileSystemRMStateStore.fs.create(tempAppAttemptFile, false);
@@ -138,10 +151,11 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
       fsOut.close();
 
       testRMAppStateStore(fsTester);
-      Assert.assertFalse(fileSystemRMStateStore.fsWorkingPath
+      Assert.assertFalse(fsTester.workingDirPathURI
           .getFileSystem(conf).exists(tempAppAttemptFile));
       testRMDTSecretManagerStateStore(fsTester);
       testCheckVersion(fsTester);
+      testAppDeletion(fsTester);
     } finally {
       cluster.shutdown();
     }

部分文件因为文件数量过多而无法显示