Browse Source

Merging r1541618 through r1542122 from trunk to branch HDFS-2832

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1542125 13f79535-47bb-0310-9956-ffa450edef68
Arpit Agarwal 11 years ago
parent
commit
132a8ff7c7
100 changed files with 3545 additions and 853 deletions
  1. 1 1
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java
  2. 0 1
      hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/AuthenticatorTestCase.java
  3. 18 1
      hadoop-common-project/hadoop-common/CHANGES.txt
  4. 11 1
      hadoop-common-project/hadoop-common/pom.xml
  5. 13 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
  6. 16 15
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CodecPool.java
  7. 12 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
  8. 186 144
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  9. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
  10. 3 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java
  11. 14 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ZKUtil.java
  12. 263 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
  13. 8 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java
  14. 8 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestZKUtil.java
  15. 1 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh
  16. 30 4
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  17. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
  18. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  19. 32 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  20. 25 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
  21. 34 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  22. 9 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
  23. 50 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java
  24. 31 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  25. 51 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  26. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java
  27. 1 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/MappableBlock.java
  28. 119 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryComputationContext.java
  29. 28 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  30. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  31. 31 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  32. 16 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  33. 38 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  34. 10 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java
  35. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  36. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
  37. 8 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  38. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java
  39. 7 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectorySnapshottable.java
  40. 15 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java
  41. 83 78
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
  42. 47 116
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/HftpFileSystem.java
  43. 177 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
  44. 17 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
  45. 15 64
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  46. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer
  47. 12 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  48. 6 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
  49. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp
  50. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/index.html
  51. 55 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
  52. 16 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
  53. 63 13
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
  54. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCorruptFilesJsp.java
  55. 44 28
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathBasedCacheRequests.java
  56. 54 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
  57. 19 45
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHftpDelegationToken.java
  58. 298 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestTokenAspect.java
  59. 36 20
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenRemoteFetcher.java
  60. 11 0
      hadoop-mapreduce-project/CHANGES.txt
  61. 88 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
  62. 1 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  63. 9 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
  64. 7 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  65. 4 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
  66. 26 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestUberAM.java
  67. 10 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
  68. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml
  69. 7 2
      hadoop-project/pom.xml
  70. 5 0
      hadoop-tools/hadoop-archives/pom.xml
  71. 5 0
      hadoop-tools/hadoop-datajoin/pom.xml
  72. 5 0
      hadoop-tools/hadoop-distcp/pom.xml
  73. 5 0
      hadoop-tools/hadoop-extras/pom.xml
  74. 5 0
      hadoop-tools/hadoop-gridmix/pom.xml
  75. 5 0
      hadoop-tools/hadoop-rumen/pom.xml
  76. 5 0
      hadoop-tools/hadoop-streaming/pom.xml
  77. 9 0
      hadoop-yarn-project/CHANGES.txt
  78. 6 0
      hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
  79. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/HAUtil.java
  80. 2 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  81. 30 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  82. 12 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
  83. 16 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
  84. 7 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMHAProtocolService.java
  85. 53 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
  86. 56 33
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
  87. 36 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreOperationFailedEvent.java
  88. 23 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreOperationFailedEventType.java
  89. 28 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/StoreFencedException.java
  90. 223 54
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
  91. 16 15
      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
  92. 48 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
  93. 121 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueuePlacementPolicy.java
  94. 200 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueuePlacementRule.java
  95. 95 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
  96. 44 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SimpleGroupsMapping.java
  97. 45 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
  98. 119 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueuePlacementPolicy.java
  99. 70 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm
  100. 1 0
      hadoop-yarn-project/hadoop-yarn/pom.xml

+ 1 - 1
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java

@@ -185,7 +185,7 @@ public class KerberosAuthenticator implements Authenticator {
       conn.setRequestMethod(AUTH_HTTP_METHOD);
       conn.connect();
       
-      if (conn.getRequestProperty(AUTHORIZATION) != null && conn.getResponseCode() == HttpURLConnection.HTTP_OK) {
+      if (conn.getResponseCode() == HttpURLConnection.HTTP_OK) {
         LOG.debug("JDK performed authentication on our behalf.");
         // If the JDK already did the SPNEGO back-and-forth for
         // us, just pull out the token.

+ 0 - 1
hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/AuthenticatorTestCase.java

@@ -136,7 +136,6 @@ public class AuthenticatorTestCase {
       TestConnectionConfigurator connConf = new TestConnectionConfigurator();
       AuthenticatedURL aUrl = new AuthenticatedURL(authenticator, connConf);
       HttpURLConnection conn = aUrl.openConnection(url, token);
-      Assert.assertTrue(token.isSet());
       Assert.assertTrue(connConf.invoked);
       String tokenStr = token.toString();
       if (doPost) {

+ 18 - 1
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -284,7 +284,7 @@ Trunk (Unreleased)
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
 
-    HADOOP-8589 ViewFs tests fail when tests and home dirs are nested (sanjay Radia)
+    HADOOP-8589. ViewFs tests fail when tests and home dirs are nested (sanjay Radia)
 
 Release 2.3.0 - UNRELEASED
 
@@ -375,6 +375,14 @@ Release 2.3.0 - UNRELEASED
     HADOOP-10064. Upgrade to maven antrun plugin version 1.7 (Arpit Agarwal via
     jeagles)
 
+    HADOOP-9594. Update apache commons math dependency (Timothy St. Clair via
+    stevel)
+
+    HADOOP-10095. In CodecPool, synchronize pool and codecList separately in
+    order to reduce lock contention.  (Nicolas Liochon via szetszwo)
+
+    HADOOP-10067. Missing POM dependency on jsr305 (Robert Rati via stevel)
+
   OPTIMIZATIONS
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
@@ -426,6 +434,9 @@ Release 2.3.0 - UNRELEASED
     HADOOP-10093. hadoop-env.cmd sets HADOOP_CLIENT_OPTS with a max heap size
     that is too small. (Shanyu Zhao via cnauroth)
 
+    HADOOP-10094. NPE in GenericOptionsParser#preProcessForWindows().
+    (Enis Soztutar via cnauroth)
+
 Release 2.2.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -476,6 +487,8 @@ Release 2.2.1 - UNRELEASED
     as [-Dkey, value] which breaks GenericsOptionParser.
     (Enis Soztutar via cnauroth)
 
+    HADOOP-10078. KerberosAuthenticator always does SPNEGO. (rkanter via tucu)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES
@@ -2298,6 +2311,10 @@ Release 0.23.10 - UNRELEASED
 
   OPTIMIZATIONS
 
+    HADOOP-9956. RPC listener inefficiently assigns connections to readers (daryn)
+
+    HADOOP-9955. RPC idle connection closing is extremely inefficient (daryn)
+
   BUG FIXES
 
 Release 0.23.9 - 2013-07-08

+ 11 - 1
hadoop-common-project/hadoop-common/pom.xml

@@ -55,7 +55,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.commons</groupId>
-      <artifactId>commons-math</artifactId>
+      <artifactId>commons-math3</artifactId>
       <scope>compile</scope>
     </dependency>
     <dependency>
@@ -83,6 +83,11 @@
       <artifactId>commons-net</artifactId>
       <scope>compile</scope>
     </dependency>
+    <dependency>
+      <groupId>commons-collections</groupId>
+      <artifactId>commons-collections</artifactId>
+      <scope>compile</scope>
+    </dependency>
     <dependency>
       <groupId>javax.servlet</groupId>
       <artifactId>servlet-api</artifactId>
@@ -213,6 +218,11 @@
       <groupId>com.jcraft</groupId>
       <artifactId>jsch</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
+      <scope>compile</scope>
+    </dependency>
 
     <dependency>
       <groupId>org.apache.zookeeper</groupId>

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

@@ -65,6 +65,13 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   /** Default value for IPC_SERVER_RPC_READ_THREADS_KEY */
   public static final int     IPC_SERVER_RPC_READ_THREADS_DEFAULT = 1;
   
+  /** Number of pending connections that may be queued per socket reader */
+  public static final String IPC_SERVER_RPC_READ_CONNECTION_QUEUE_SIZE_KEY =
+      "ipc.server.read.connection-queue.size";
+  /** Default value for IPC_SERVER_RPC_READ_CONNECTION_QUEUE_SIZE */
+  public static final int IPC_SERVER_RPC_READ_CONNECTION_QUEUE_SIZE_DEFAULT =
+      100;
+      
   public static final String IPC_MAXIMUM_DATA_LENGTH =
       "ipc.maximum.data.length";
   
@@ -219,4 +226,10 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   public static final String  IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY = "ipc.client.fallback-to-simple-auth-allowed";
   public static final boolean IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT = false;
 
+  /** How often the server scans for idle connections */
+  public static final String IPC_CLIENT_CONNECTION_IDLESCANINTERVAL_KEY =
+      "ipc.client.connection.idle-scan-interval.ms";
+  /** Default value for IPC_SERVER_CONNECTION_IDLE_SCAN_INTERVAL_KEY */
+  public static final int IPC_CLIENT_CONNECTION_IDLESCANINTERVAL_DEFAULT =
+      10000;
 }

+ 16 - 15
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CodecPool.java

@@ -85,16 +85,15 @@ public class CodecPool {
     T codec = null;
     
     // Check if an appropriate codec is available
+    List<T> codecList;
     synchronized (pool) {
-      if (pool.containsKey(codecClass)) {
-        List<T> codecList = pool.get(codecClass);
-        
-        if (codecList != null) {
-          synchronized (codecList) {
-            if (!codecList.isEmpty()) {
-              codec = codecList.remove(codecList.size()-1);
-            }
-          }
+      codecList = pool.get(codecClass);
+    }
+
+    if (codecList != null) {
+      synchronized (codecList) {
+        if (!codecList.isEmpty()) {
+          codec = codecList.remove(codecList.size() - 1);
         }
       }
     }
@@ -105,15 +104,17 @@ public class CodecPool {
   private static <T> void payback(Map<Class<T>, List<T>> pool, T codec) {
     if (codec != null) {
       Class<T> codecClass = ReflectionUtils.getClass(codec);
+      List<T> codecList;
       synchronized (pool) {
-        if (!pool.containsKey(codecClass)) {
-          pool.put(codecClass, new ArrayList<T>());
+        codecList = pool.get(codecClass);
+        if (codecList == null) {
+          codecList = new ArrayList<T>();
+          pool.put(codecClass, codecList);
         }
+      }
 
-        List<T> codecList = pool.get(codecClass);
-        synchronized (codecList) {
-          codecList.add(codec);
-        }
+      synchronized (codecList) {
+        codecList.add(codec);
       }
     }
   }

+ 12 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java

@@ -114,7 +114,17 @@ public class NativeIO {
     public static interface CacheTracker {
       public void fadvise(String identifier, long offset, long len, int flags);
     }
-    
+
+    public static CacheManipulator cacheManipulator = new CacheManipulator();
+
+    @VisibleForTesting
+    public static class CacheManipulator {
+      public void mlock(String identifier, ByteBuffer buffer,
+          long len) throws IOException {
+        POSIX.mlock(buffer, len);
+      }
+    }
+
     static {
       if (NativeCodeLoader.isNativeCodeLoaded()) {
         try {
@@ -249,7 +259,7 @@ public class NativeIO {
      * 
      * @throws NativeIOException
      */
-    public static void mlock(ByteBuffer buffer, long len)
+    static void mlock(ByteBuffer buffer, long len)
         throws IOException {
       assertCodeLoaded();
       if (!buffer.isDirect()) {

+ 186 - 144
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -51,11 +51,13 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Random;
 import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import javax.security.sasl.Sasl;
 import javax.security.sasl.SaslException;
@@ -345,17 +347,8 @@ public abstract class Server {
   private int port;                               // port we listen on
   private int handlerCount;                       // number of handler threads
   private int readThreads;                        // number of read threads
+  private int readerPendingConnectionQueue;         // number of connections to queue per read thread
   private Class<? extends Writable> rpcRequestClass;   // class used for deserializing the rpc request
-  private int maxIdleTime;                        // the maximum idle time after 
-                                                  // which a client may be disconnected
-  private int thresholdIdleConnections;           // the number of idle connections
-                                                  // after which we will start
-                                                  // cleaning up idle 
-                                                  // connections
-  int maxConnectionsToNuke;                       // the max number of 
-                                                  // connections to nuke
-                                                  //during a cleanup
-  
   protected RpcMetrics rpcMetrics;
   protected RpcDetailedMetrics rpcDetailedMetrics;
   
@@ -373,13 +366,10 @@ public abstract class Server {
   volatile private boolean running = true;         // true while server runs
   private BlockingQueue<Call> callQueue; // queued calls
 
-  private List<Connection> connectionList = 
-    Collections.synchronizedList(new LinkedList<Connection>());
-  //maintain a list
-  //of client connections
+  // maintains the set of client connections and handles idle timeouts
+  private ConnectionManager connectionManager;
   private Listener listener = null;
   private Responder responder = null;
-  private int numConnections = 0;
   private Handler[] handlers = null;
 
   /**
@@ -449,8 +439,8 @@ public abstract class Server {
   }
 
   @VisibleForTesting
-  List<Connection> getConnections() {
-    return connectionList;
+  Connection[] getConnections() {
+    return connectionManager.toArray();
   }
 
   /**
@@ -518,11 +508,6 @@ public abstract class Server {
     private Reader[] readers = null;
     private int currentReader = 0;
     private InetSocketAddress address; //the address we bind at
-    private Random rand = new Random();
-    private long lastCleanupRunTime = 0; //the last time when a cleanup connec-
-                                         //-tion (for idle connections) ran
-    private long cleanupInterval = 10000; //the minimum interval between 
-                                          //two cleanup runs
     private int backlogLength = conf.getInt(
         CommonConfigurationKeysPublic.IPC_SERVER_LISTEN_QUEUE_SIZE_KEY,
         CommonConfigurationKeysPublic.IPC_SERVER_LISTEN_QUEUE_SIZE_DEFAULT);
@@ -553,12 +538,14 @@ public abstract class Server {
     }
     
     private class Reader extends Thread {
-      private volatile boolean adding = false;
+      final private BlockingQueue<Connection> pendingConnections;
       private final Selector readSelector;
 
       Reader(String name) throws IOException {
         super(name);
 
+        this.pendingConnections =
+            new LinkedBlockingQueue<Connection>(readerPendingConnectionQueue);
         this.readSelector = Selector.open();
       }
       
@@ -580,10 +567,14 @@ public abstract class Server {
         while (running) {
           SelectionKey key = null;
           try {
+            // consume as many connections as currently queued to avoid
+            // unbridled acceptance of connections that starves the select
+            int size = pendingConnections.size();
+            for (int i=size; i>0; i--) {
+              Connection conn = pendingConnections.take();
+              conn.channel.register(readSelector, SelectionKey.OP_READ, conn);
+            }
             readSelector.select();
-            while (adding) {
-              this.wait(1000);
-            }              
 
             Iterator<SelectionKey> iter = readSelector.selectedKeys().iterator();
             while (iter.hasNext()) {
@@ -607,26 +598,14 @@ public abstract class Server {
       }
 
       /**
-       * This gets reader into the state that waits for the new channel
-       * to be registered with readSelector. If it was waiting in select()
-       * the thread will be woken up, otherwise whenever select() is called
-       * it will return even if there is nothing to read and wait
-       * in while(adding) for finishAdd call
+       * Updating the readSelector while it's being used is not thread-safe,
+       * so the connection must be queued.  The reader will drain the queue
+       * and update its readSelector before performing the next select
        */
-      public void startAdd() {
-        adding = true;
+      public void addConnection(Connection conn) throws InterruptedException {
+        pendingConnections.put(conn);
         readSelector.wakeup();
       }
-      
-      public synchronized SelectionKey registerChannel(SocketChannel channel)
-                                                          throws IOException {
-          return channel.register(readSelector, SelectionKey.OP_READ);
-      }
-
-      public synchronized void finishAdd() {
-        adding = false;
-        this.notify();        
-      }
 
       void shutdown() {
         assert !running;
@@ -638,58 +617,12 @@ public abstract class Server {
         }
       }
     }
-    /** cleanup connections from connectionList. Choose a random range
-     * to scan and also have a limit on the number of the connections
-     * that will be cleanedup per run. The criteria for cleanup is the time
-     * for which the connection was idle. If 'force' is true then all 
-     * connections will be looked at for the cleanup.
-     */
-    private void cleanupConnections(boolean force) {
-      if (force || numConnections > thresholdIdleConnections) {
-        long currentTime = Time.now();
-        if (!force && (currentTime - lastCleanupRunTime) < cleanupInterval) {
-          return;
-        }
-        int start = 0;
-        int end = numConnections - 1;
-        if (!force) {
-          start = rand.nextInt() % numConnections;
-          end = rand.nextInt() % numConnections;
-          int temp;
-          if (end < start) {
-            temp = start;
-            start = end;
-            end = temp;
-          }
-        }
-        int i = start;
-        int numNuked = 0;
-        while (i <= end) {
-          Connection c;
-          synchronized (connectionList) {
-            try {
-              c = connectionList.get(i);
-            } catch (Exception e) {return;}
-          }
-          if (c.timedOut(currentTime)) {
-            if (LOG.isDebugEnabled())
-              LOG.debug(getName() + ": disconnecting client " + c.getHostAddress());
-            closeConnection(c);
-            numNuked++;
-            end--;
-            c = null;
-            if (!force && numNuked == maxConnectionsToNuke) break;
-          }
-          else i++;
-        }
-        lastCleanupRunTime = Time.now();
-      }
-    }
 
     @Override
     public void run() {
       LOG.info(getName() + ": starting");
       SERVER.set(Server.this);
+      connectionManager.startIdleScan();
       while (running) {
         SelectionKey key = null;
         try {
@@ -713,12 +646,11 @@ public abstract class Server {
           // some thread(s) a chance to finish
           LOG.warn("Out of Memory in server select", e);
           closeCurrentConnection(key, e);
-          cleanupConnections(true);
+          connectionManager.closeIdle(true);
           try { Thread.sleep(60000); } catch (Exception ie) {}
         } catch (Exception e) {
           closeCurrentConnection(key, e);
         }
-        cleanupConnections(false);
       }
       LOG.info("Stopping " + this.getName());
 
@@ -731,10 +663,9 @@ public abstract class Server {
         selector= null;
         acceptChannel= null;
         
-        // clean up all connections
-        while (!connectionList.isEmpty()) {
-          closeConnection(connectionList.remove(0));
-        }
+        // close all connections
+        connectionManager.stopIdleScan();
+        connectionManager.closeAll();
       }
     }
 
@@ -742,8 +673,6 @@ public abstract class Server {
       if (key != null) {
         Connection c = (Connection)key.attachment();
         if (c != null) {
-          if (LOG.isDebugEnabled())
-            LOG.debug(getName() + ": disconnecting client " + c.getHostAddress());
           closeConnection(c);
           c = null;
         }
@@ -754,8 +683,7 @@ public abstract class Server {
       return (InetSocketAddress)acceptChannel.socket().getLocalSocketAddress();
     }
     
-    void doAccept(SelectionKey key) throws IOException,  OutOfMemoryError {
-      Connection c = null;
+    void doAccept(SelectionKey key) throws InterruptedException, IOException,  OutOfMemoryError {
       ServerSocketChannel server = (ServerSocketChannel) key.channel();
       SocketChannel channel;
       while ((channel = server.accept()) != null) {
@@ -765,22 +693,9 @@ public abstract class Server {
         channel.socket().setKeepAlive(true);
         
         Reader reader = getReader();
-        try {
-          reader.startAdd();
-          SelectionKey readKey = reader.registerChannel(channel);
-          c = new Connection(readKey, channel, Time.now());
-          readKey.attach(c);
-          synchronized (connectionList) {
-            connectionList.add(numConnections, c);
-            numConnections++;
-          }
-          if (LOG.isDebugEnabled())
-            LOG.debug("Server connection from " + c.toString() +
-                "; # active connections: " + numConnections +
-                "; # queued calls: " + callQueue.size());          
-        } finally {
-          reader.finishAdd(); 
-        }
+        Connection c = connectionManager.register(channel);
+        key.attach(c);  // so closeCurrentConnection can get the object
+        reader.addConnection(c);
       }
     }
 
@@ -808,10 +723,6 @@ public abstract class Server {
         count = -1; //so that the (count < 0) block is executed
       }
       if (count < 0) {
-        if (LOG.isDebugEnabled())
-          LOG.debug(getName() + ": disconnecting client " + 
-                    c + ". Number of active connections: "+
-                    numConnections);
         closeConnection(c);
         c = null;
       }
@@ -1190,8 +1101,7 @@ public abstract class Server {
     private boolean sentNegotiate = false;
     private boolean useWrap = false;
     
-    public Connection(SelectionKey key, SocketChannel channel, 
-                      long lastContact) {
+    public Connection(SocketChannel channel, long lastContact) {
       this.channel = channel;
       this.lastContact = lastContact;
       this.data = null;
@@ -1253,12 +1163,6 @@ public abstract class Server {
       rpcCount++;
     }
     
-    private boolean timedOut(long currentTime) {
-      if (isIdle() && currentTime -  lastContact > maxIdleTime)
-        return true;
-      return false;
-    }
-    
     private UserGroupInformation getAuthorizedUgi(String authorizedId)
         throws InvalidToken, AccessControlException {
       if (authMethod == AuthMethod.TOKEN) {
@@ -2189,16 +2093,10 @@ public abstract class Server {
           CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_KEY,
           CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_DEFAULT);
     }
+    this.readerPendingConnectionQueue = conf.getInt(
+        CommonConfigurationKeys.IPC_SERVER_RPC_READ_CONNECTION_QUEUE_SIZE_KEY,
+        CommonConfigurationKeys.IPC_SERVER_RPC_READ_CONNECTION_QUEUE_SIZE_DEFAULT);
     this.callQueue  = new LinkedBlockingQueue<Call>(maxQueueSize); 
-    this.maxIdleTime = 2 * conf.getInt(
-        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
-        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_DEFAULT);
-    this.maxConnectionsToNuke = conf.getInt(
-        CommonConfigurationKeysPublic.IPC_CLIENT_KILL_MAX_KEY,
-        CommonConfigurationKeysPublic.IPC_CLIENT_KILL_MAX_DEFAULT);
-    this.thresholdIdleConnections = conf.getInt(
-        CommonConfigurationKeysPublic.IPC_CLIENT_IDLETHRESHOLD_KEY,
-        CommonConfigurationKeysPublic.IPC_CLIENT_IDLETHRESHOLD_DEFAULT);
     this.secretManager = (SecretManager<TokenIdentifier>) secretManager;
     this.authorize = 
       conf.getBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, 
@@ -2219,6 +2117,7 @@ public abstract class Server {
 
     // Create the responder here
     responder = new Responder();
+    connectionManager = new ConnectionManager();
     
     if (secretManager != null) {
       SaslRpcServer.init(conf);
@@ -2277,11 +2176,7 @@ public abstract class Server {
   }
   
   private void closeConnection(Connection connection) {
-    synchronized (connectionList) {
-      if (connectionList.remove(connection))
-        numConnections--;
-    }
-    connection.close();
+    connectionManager.close(connection);
   }
   
   /**
@@ -2536,7 +2431,7 @@ public abstract class Server {
    * @return the number of open rpc connections
    */
   public int getNumOpenConnections() {
-    return numConnections;
+    return connectionManager.size();
   }
   
   /**
@@ -2646,4 +2541,151 @@ public abstract class Server {
     int nBytes = initialRemaining - buf.remaining(); 
     return (nBytes > 0) ? nBytes : ret;
   }
+  
+  private class ConnectionManager {
+    final private AtomicInteger count = new AtomicInteger();    
+    final private Set<Connection> connections;
+
+    final private Timer idleScanTimer;
+    final private int idleScanThreshold;
+    final private int idleScanInterval;
+    final private int maxIdleTime;
+    final private int maxIdleToClose;
+    
+    ConnectionManager() {
+      this.idleScanTimer = new Timer(
+          "IPC Server idle connection scanner for port " + getPort(), true);
+      this.idleScanThreshold = conf.getInt(
+          CommonConfigurationKeysPublic.IPC_CLIENT_IDLETHRESHOLD_KEY,
+          CommonConfigurationKeysPublic.IPC_CLIENT_IDLETHRESHOLD_DEFAULT);
+      this.idleScanInterval = conf.getInt(
+          CommonConfigurationKeys.IPC_CLIENT_CONNECTION_IDLESCANINTERVAL_KEY,
+          CommonConfigurationKeys.IPC_CLIENT_CONNECTION_IDLESCANINTERVAL_DEFAULT);
+      this.maxIdleTime = 2 * conf.getInt(
+          CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
+          CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_DEFAULT);
+      this.maxIdleToClose = conf.getInt(
+          CommonConfigurationKeysPublic.IPC_CLIENT_KILL_MAX_KEY,
+          CommonConfigurationKeysPublic.IPC_CLIENT_KILL_MAX_DEFAULT);
+      // create a set with concurrency -and- a thread-safe iterator, add 2
+      // for listener and idle closer threads
+      this.connections = Collections.newSetFromMap(
+          new ConcurrentHashMap<Connection,Boolean>(
+              maxQueueSize, 0.75f, readThreads+2));
+    }
+
+    private boolean add(Connection connection) {
+      boolean added = connections.add(connection);
+      if (added) {
+        count.getAndIncrement();
+      }
+      return added;
+    }
+    
+    private boolean remove(Connection connection) {
+      boolean removed = connections.remove(connection);
+      if (removed) {
+        count.getAndDecrement();
+      }
+      return removed;
+    }
+    
+    int size() {
+      return count.get();
+    }
+
+    Connection[] toArray() {
+      return connections.toArray(new Connection[0]);
+    }
+
+    Connection register(SocketChannel channel) {
+      Connection connection = new Connection(channel, Time.now());
+      add(connection);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Server connection from " + connection +
+            "; # active connections: " + size() +
+            "; # queued calls: " + callQueue.size());
+      }      
+      return connection;
+    }
+    
+    boolean close(Connection connection) {
+      boolean exists = remove(connection);
+      if (exists) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(Thread.currentThread().getName() +
+              ": disconnecting client " + connection +
+              ". Number of active connections: "+ size());
+        }
+        // only close if actually removed to avoid double-closing due
+        // to possible races
+        connection.close();
+      }
+      return exists;
+    }
+    
+    // synch'ed to avoid explicit invocation upon OOM from colliding with
+    // timer task firing
+    synchronized void closeIdle(boolean scanAll) {
+      long minLastContact = Time.now() - maxIdleTime;
+      // concurrent iterator might miss new connections added
+      // during the iteration, but that's ok because they won't
+      // be idle yet anyway and will be caught on next scan
+      int closed = 0;
+      for (Connection connection : connections) {
+        // stop if connections dropped below threshold unless scanning all
+        if (!scanAll && size() < idleScanThreshold) {
+          break;
+        }
+        // stop if not scanning all and max connections are closed
+        if (connection.isIdle() &&
+            connection.getLastContact() < minLastContact &&
+            close(connection) &&
+            !scanAll && (++closed == maxIdleToClose)) {
+          break;
+        }
+      }
+    }
+    
+    void closeAll() {
+      // use a copy of the connections to be absolutely sure the concurrent
+      // iterator doesn't miss a connection
+      for (Connection connection : toArray()) {
+        close(connection);
+      }
+    }
+    
+    void startIdleScan() {
+      scheduleIdleScanTask();
+    }
+    
+    void stopIdleScan() {
+      idleScanTimer.cancel();
+    }
+    
+    private void scheduleIdleScanTask() {
+      if (!running) {
+        return;
+      }
+      TimerTask idleScanTask = new TimerTask(){
+        @Override
+        public void run() {
+          if (!running) {
+            return;
+          }
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(Thread.currentThread().getName()+": task running");
+          }
+          try {
+            closeIdle(false);
+          } finally {
+            // explicitly reschedule so next execution occurs relative
+            // to the end of this scan, not the beginning
+            scheduleIdleScanTask();
+          }
+        }
+      };
+      idleScanTimer.schedule(idleScanTask, idleScanInterval);
+    }
+  }
 }

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java

@@ -38,7 +38,7 @@ import static com.google.common.base.Preconditions.*;
 import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.commons.math.util.MathUtils;
+import org.apache.commons.math3.util.ArithmeticUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.metrics2.MetricsInfo;
 import org.apache.hadoop.metrics2.MetricsCollector;
@@ -457,7 +457,7 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
       MetricsConfig conf = entry.getValue();
       int sinkPeriod = conf.getInt(PERIOD_KEY, PERIOD_DEFAULT);
       confPeriod = confPeriod == 0 ? sinkPeriod
-                                   : MathUtils.gcd(confPeriod, sinkPeriod);
+                                   : ArithmeticUtils.gcd(confPeriod, sinkPeriod);
       String clsName = conf.getClassName("");
       if (clsName == null) continue;  // sink can be registered later on
       String sinkName = entry.getKey();

+ 3 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java

@@ -431,6 +431,9 @@ public class GenericOptionsParser {
     if (!Shell.WINDOWS) {
       return args;
     }
+    if (args == null) {
+      return null;
+    }
     List<String> newArgs = new ArrayList<String>(args.length);
     for (int i=0; i < args.length; i++) {
       String prop = null;

+ 14 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ZKUtil.java

@@ -71,6 +71,20 @@ public class ZKUtil {
     return perm;
   }
 
+  /**
+   * Helper method to remove a subset of permissions (remove) from a
+   * given set (perms).
+   * @param perms The permissions flag to remove from. Should be an OR of a
+   *              some combination of {@link ZooDefs.Perms}
+   * @param remove The permissions to be removed. Should be an OR of a
+   *              some combination of {@link ZooDefs.Perms}
+   * @return A permissions flag that is an OR of {@link ZooDefs.Perms}
+   * present in perms and not present in remove
+   */
+  public static int removeSpecificPerms(int perms, int remove) {
+    return perms ^ remove;
+  }
+
   /**
    * Parse comma separated list of ACL entries to secure generated nodes, e.g.
    * <code>sasl:hdfs/host1@MY.DOMAIN:cdrwa,sasl:hdfs/host2@MY.DOMAIN:cdrwa</code>

+ 263 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java

@@ -44,12 +44,19 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Random;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import javax.net.SocketFactory;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IntWritable;
@@ -64,8 +71,10 @@ import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
 import org.apache.hadoop.net.ConnectTimeoutException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Assume;
+import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
@@ -79,7 +88,7 @@ public class TestIPC {
   public static final Log LOG =
     LogFactory.getLog(TestIPC.class);
   
-  final private static Configuration conf = new Configuration();
+  private static Configuration conf;
   final static private int PING_INTERVAL = 1000;
   final static private int MIN_SLEEP_TIME = 1000;
   /**
@@ -89,7 +98,9 @@ public class TestIPC {
   static boolean WRITABLE_FAULTS_ENABLED = true;
   static int WRITABLE_FAULTS_SLEEP = 0;
   
-  static {
+  @Before
+  public void setupConf() {
+    conf = new Configuration();
     Client.setPingInterval(conf, PING_INTERVAL);
   }
 
@@ -613,6 +624,255 @@ public class TestIPC {
     server.stop();
   }
 
+  private static class TestServerQueue extends Server {
+    final CountDownLatch firstCallLatch = new CountDownLatch(1);
+    final CountDownLatch callBlockLatch = new CountDownLatch(1);
+    
+    TestServerQueue(int expectedCalls, int readers, int callQ, int handlers,
+        Configuration conf) throws IOException {
+      super(ADDRESS, 0, LongWritable.class, handlers, readers, callQ, conf, null, null); 
+    }
+
+    @Override
+    public Writable call(RPC.RpcKind rpcKind, String protocol, Writable param,
+        long receiveTime) throws IOException {
+      firstCallLatch.countDown();
+      try {
+        callBlockLatch.await();
+      } catch (InterruptedException e) {
+        throw new IOException(e);
+      }
+      return param;
+    }
+  }
+
+  /**
+   * Check that reader queueing works
+   * @throws BrokenBarrierException 
+   * @throws InterruptedException 
+   */
+  @Test(timeout=60000)
+  public void testIpcWithReaderQueuing() throws Exception {
+    // 1 reader, 1 connectionQ slot, 1 callq
+    for (int i=0; i < 10; i++) {
+      checkBlocking(1, 1, 1);
+    }
+    // 4 readers, 5 connectionQ slots, 2 callq
+    for (int i=0; i < 10; i++) {
+      checkBlocking(4, 5, 2);
+    }
+  }
+  
+  // goal is to jam a handler with a connection, fill the callq with
+  // connections, in turn jamming the readers - then flood the server and
+  // ensure that the listener blocks when the reader connection queues fill
+  private void checkBlocking(int readers, int readerQ, int callQ) throws Exception {
+    int handlers = 1; // makes it easier
+    
+    final Configuration conf = new Configuration();
+    conf.setInt(CommonConfigurationKeys.IPC_SERVER_RPC_READ_CONNECTION_QUEUE_SIZE_KEY, readerQ);
+
+    // send in enough clients to block up the handlers, callq, and readers
+    int initialClients = readers + callQ + handlers;
+    // max connections we should ever end up accepting at once
+    int maxAccept = initialClients + readers*readerQ + 1; // 1 = listener
+    // stress it with 2X the max
+    int clients = maxAccept*2;
+    
+    final AtomicInteger failures = new AtomicInteger(0);
+    final CountDownLatch callFinishedLatch = new CountDownLatch(clients);
+
+    // start server
+    final TestServerQueue server =
+        new TestServerQueue(clients, readers, callQ, handlers, conf);
+    final InetSocketAddress addr = NetUtils.getConnectAddress(server);
+    server.start();
+
+    Client.setConnectTimeout(conf, 10000);
+    
+    // instantiate the threads, will start in batches
+    Thread[] threads = new Thread[clients];
+    for (int i=0; i<clients; i++) {
+      threads[i] = new Thread(new Runnable() {
+        @Override
+        public void run() {
+          Client client = new Client(LongWritable.class, conf);
+          try {
+            client.call(new LongWritable(Thread.currentThread().getId()),
+                addr, null, null, 60000, conf);
+          } catch (Throwable e) {
+            LOG.error(e);
+            failures.incrementAndGet();
+            return;
+          } finally {
+            callFinishedLatch.countDown();            
+            client.stop();
+          }
+        }
+      });
+    }
+    
+    // start enough clients to block up the handler, callq, and each reader;
+    // let the calls sequentially slot in to avoid some readers blocking
+    // and others not blocking in the race to fill the callq
+    for (int i=0; i < initialClients; i++) {
+      threads[i].start();
+      if (i==0) {
+        // let first reader block in a call
+        server.firstCallLatch.await();
+      } else if (i <= callQ) {
+        // let subsequent readers jam the callq, will happen immediately 
+        while (server.getCallQueueLen() != i) {
+          Thread.sleep(1);
+        }
+      } // additional threads block the readers trying to add to the callq
+    }
+
+    // wait till everything is slotted, should happen immediately
+    Thread.sleep(10);
+    if (server.getNumOpenConnections() < initialClients) {
+      LOG.info("(initial clients) need:"+initialClients+" connections have:"+server.getNumOpenConnections());
+      Thread.sleep(100);
+    }
+    LOG.info("ipc layer should be blocked");
+    assertEquals(callQ, server.getCallQueueLen());
+    assertEquals(initialClients, server.getNumOpenConnections());
+    
+    // now flood the server with the rest of the connections, the reader's
+    // connection queues should fill and then the listener should block
+    for (int i=initialClients; i<clients; i++) {
+      threads[i].start();
+    }
+    Thread.sleep(10);
+    if (server.getNumOpenConnections() < maxAccept) {
+      LOG.info("(max clients) need:"+maxAccept+" connections have:"+server.getNumOpenConnections());
+      Thread.sleep(100);
+    }
+    // check a few times to make sure we didn't go over
+    for (int i=0; i<4; i++) {
+      assertEquals(maxAccept, server.getNumOpenConnections());
+      Thread.sleep(100);
+    }
+    
+    // sanity check that no calls have finished
+    assertEquals(clients, callFinishedLatch.getCount());
+    LOG.info("releasing the calls");
+    server.callBlockLatch.countDown();
+    callFinishedLatch.await();
+    for (Thread t : threads) {
+      t.join();
+    }
+    assertEquals(0, failures.get());
+    server.stop();
+  }
+
+  @Test(timeout=30000)
+  public void testConnectionIdleTimeouts() throws Exception {
+    ((Log4JLogger)Server.LOG).getLogger().setLevel(Level.DEBUG);
+    final int maxIdle = 1000;
+    final int cleanupInterval = maxIdle*3/4; // stagger cleanups
+    final int killMax = 3;
+    final int clients = 1 + killMax*2; // 1 to block, 2 batches to kill
+    
+    conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, maxIdle);
+    conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_IDLETHRESHOLD_KEY, 0);
+    conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_KILL_MAX_KEY, killMax);
+    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECTION_IDLESCANINTERVAL_KEY, cleanupInterval);
+    
+    final CyclicBarrier firstCallBarrier = new CyclicBarrier(2);
+    final CyclicBarrier callBarrier = new CyclicBarrier(clients);
+    final CountDownLatch allCallLatch = new CountDownLatch(clients);
+    final AtomicBoolean error = new AtomicBoolean();
+    
+    final TestServer server = new TestServer(clients, false);
+    Thread[] threads = new Thread[clients];
+    try {
+      server.callListener = new Runnable(){
+        AtomicBoolean first = new AtomicBoolean(true);
+        @Override
+        public void run() {
+          try {
+            allCallLatch.countDown();
+            // block first call
+            if (first.compareAndSet(true, false)) {
+              firstCallBarrier.await();
+            } else {
+              callBarrier.await();
+            }
+          } catch (Throwable t) {
+            LOG.error(t);
+            error.set(true); 
+          } 
+        }
+      };
+      server.start();
+
+      // start client
+      final CountDownLatch callReturned = new CountDownLatch(clients-1);
+      final InetSocketAddress addr = NetUtils.getConnectAddress(server);
+      final Configuration clientConf = new Configuration();
+      clientConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, 10000);
+      for (int i=0; i < clients; i++) {
+        threads[i] = new Thread(new Runnable(){
+          @Override
+          public void run() {
+            Client client = new Client(LongWritable.class, clientConf);
+            try {
+              client.call(new LongWritable(Thread.currentThread().getId()),
+                  addr, null, null, 0, clientConf);
+              callReturned.countDown();
+              Thread.sleep(10000);
+            } catch (IOException e) {
+              LOG.error(e);
+            } catch (InterruptedException e) {
+            }
+          }
+        });
+        threads[i].start();
+      }
+      
+      // all calls blocked in handler so all connections made
+      allCallLatch.await();
+      assertFalse(error.get());
+      assertEquals(clients, server.getNumOpenConnections());
+      
+      // wake up blocked calls and wait for client call to return, no
+      // connections should have closed
+      callBarrier.await();
+      callReturned.await();
+      assertEquals(clients, server.getNumOpenConnections());
+      
+      // server won't close till maxIdle*2, so give scanning thread time to
+      // be almost ready to close idle connection.  after which it should
+      // close max connections on every cleanupInterval
+      Thread.sleep(maxIdle*2-cleanupInterval);
+      for (int i=clients; i > 1; i -= killMax) {
+        Thread.sleep(cleanupInterval);
+        assertFalse(error.get());
+        assertEquals(i, server.getNumOpenConnections());
+      }
+
+      // connection for the first blocked call should still be open
+      Thread.sleep(cleanupInterval);
+      assertFalse(error.get());
+      assertEquals(1, server.getNumOpenConnections());
+     
+      // wake up call and ensure connection times out
+      firstCallBarrier.await();
+      Thread.sleep(maxIdle*2);
+      assertFalse(error.get());
+      assertEquals(0, server.getNumOpenConnections());
+    } finally {
+      for (Thread t : threads) {
+        if (t != null) {
+          t.interrupt();
+          t.join();
+        }
+        server.stop();
+      }
+    }
+  }
+  
   /**
    * Make a call from a client and verify if header info is changed in server side
    */
@@ -622,7 +882,7 @@ public class TestIPC {
 
     client.call(new LongWritable(RANDOM.nextLong()),
         addr, null, null, MIN_SLEEP_TIME, serviceClass, conf);
-    Connection connection = server.getConnections().get(0);
+    Connection connection = server.getConnections()[0];
     int serviceClass2 = connection.getServiceClass();
     assertFalse(noChanged ^ serviceClass == serviceClass2);
     client.stop();

+ 8 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java

@@ -282,4 +282,12 @@ public class TestGenericOptionsParser extends TestCase {
       Arrays.toString(remainingArgs) + Arrays.toString(expectedRemainingArgs),
       expectedRemainingArgs, remainingArgs);
   }
+
+  /** Test passing null as args. Some classes still call
+   * Tool interface from java passing null.
+   */
+  public void testNullArgs() throws IOException {
+    GenericOptionsParser parser = new GenericOptionsParser(conf, null);
+    parser.getRemainingArgs();
+  }
 }

+ 8 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestZKUtil.java

@@ -24,7 +24,6 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.hadoop.util.ZKUtil;
 import org.apache.hadoop.util.ZKUtil.BadAclFormatException;
 import org.apache.hadoop.util.ZKUtil.ZKAuthInfo;
 import org.apache.zookeeper.ZooDefs.Perms;
@@ -75,6 +74,14 @@ public class TestZKUtil {
     }
   }
 
+  @Test
+  public void testRemoveSpecificPerms() {
+    int perms = Perms.ALL;
+    int remove = Perms.CREATE;
+    int newPerms = ZKUtil.removeSpecificPerms(perms, remove);
+    assertEquals("Removal failed", 0, newPerms & Perms.CREATE);
+  }
+
   @Test
   public void testGoodACLs() {
     List<ACL> result = ZKUtil.parseACLs(

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh

@@ -55,7 +55,7 @@ print "Setting HTTPFS_HOME:          ${HTTPFS_HOME}"
 #
 if [ -e "${HTTPFS_HOME}/bin/httpfs-env.sh" ]; then
   print "Sourcing:                    ${HTTPFS_HOME}/bin/httpfs-env.sh"
-  source ${HTTPFS_HOME}/bin/HTTPFS-env.sh
+  source ${HTTPFS_HOME}/bin/httpfs-env.sh
   grep "^ *export " ${HTTPFS_HOME}/bin/httpfs-env.sh | sed 's/ *export/  setting/'
 fi
 

+ 30 - 4
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -13,6 +13,9 @@ Trunk (Unreleased)
 
     HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)
 
+    HDFS-5444. Choose default web UI based on browser capabilities. (Haohui Mai
+    via jing9)
+
   IMPROVEMENTS
 
     HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.
@@ -194,6 +197,8 @@ Trunk (Unreleased)
 
     HDFS-5485. Add command-line support for modifyDirective. (cmccabe)
 
+    HDFS-5366. recaching improvements (cmccabe)
+
   OPTIMIZATIONS
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
 
@@ -475,6 +480,20 @@ Release 2.3.0 - UNRELEASED
     HDFS-5495. Remove further JUnit3 usages from HDFS.
     (Jarek Jarcec Cecho via wang)
 
+    HDFS-5325. Remove WebHdfsFileSystem#ConnRunner. (Haohui Mai via jing9) 
+
+    HDFS-5488. Clean up TestHftpURLTimeout. (Haohui Mai via jing9)
+
+    HDFS-5440. Extract the logic of handling delegation tokens in HftpFileSystem 
+    to the TokenAspect class. (Haohui Mai via jing9)
+
+    HDFS-5487. Introduce unit test for TokenAspect. (Haohui Mai via jing9)
+
+    HDFS-4995. Make getContentSummary less expensive. (kihwal)
+
+    HDFS-5506. Use URLConnectionFactory in DelegationTokenFetcher. (Haohui Mai
+    via jing9)
+
   OPTIMIZATIONS
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
@@ -529,13 +548,20 @@ Release 2.3.0 - UNRELEASED
     HDFS-5476. Snapshot: clean the blocks/files/directories under a renamed 
     file/directory while deletion. (jing9)
 
-    HDFS-5325. Remove WebHdfsFileSystem#ConnRunner. (Haohui Mai via jing9) 
-
-    HDFS-5488. Clean up TestHftpURLTimeout. (Haohui Mai via jing9)
-
     HDFS-5425. Renaming underconstruction file with snapshots can make NN failure on 
     restart. (jing9 and Vinay)
 
+    HDFS-5474. Deletesnapshot can make Namenode in safemode on NN restarts. 
+    (Sathish via jing9)
+
+    HDFS-5075. httpfs-config.sh calls out incorrect env script name
+    (Timothy St. Clair via stevel)
+
+    HDFS-5504. In HA mode, OP_DELETE_SNAPSHOT is not decrementing the safemode threshold, 
+    leads to NN safemode. (Vinay via jing9)
+
+    HDFS-5438. Flaws in block report processing can cause data loss. (kihwal)
+
 Release 2.2.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java

@@ -42,4 +42,8 @@ public class DFSClientFaultInjector {
   public boolean uncorruptPacket() {
     return false;
   }
+
+  public boolean failPacket() {
+    return false;
+  }
 }

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

@@ -198,6 +198,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   
   public static final String  DFS_LIST_LIMIT = "dfs.ls.limit";
   public static final int     DFS_LIST_LIMIT_DEFAULT = 1000;
+  public static final String  DFS_CONTENT_SUMMARY_LIMIT_KEY = "dfs.content-summary.limit";
+  public static final int     DFS_CONTENT_SUMMARY_LIMIT_DEFAULT = 0;
   public static final String  DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY = "dfs.datanode.failed.volumes.tolerated";
   public static final int     DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT = 0;
   public static final String  DFS_DATANODE_SYNCONCLOSE_KEY = "dfs.datanode.synconclose";
@@ -342,6 +344,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final boolean DFS_DATANODE_TRANSFERTO_ALLOWED_DEFAULT = true;
   public static final String  DFS_HEARTBEAT_INTERVAL_KEY = "dfs.heartbeat.interval";
   public static final long    DFS_HEARTBEAT_INTERVAL_DEFAULT = 3;
+  public static final String  DFS_NAMENODE_PATH_BASED_CACHE_RETRY_INTERVAL_MS = "dfs.namenode.path.based.cache.retry.interval.ms";
+  public static final long    DFS_NAMENODE_PATH_BASED_CACHE_RETRY_INTERVAL_MS_DEFAULT = 60000L;
   public static final String  DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY = "dfs.namenode.decommission.interval";
   public static final int     DFS_NAMENODE_DECOMMISSION_INTERVAL_DEFAULT = 30;
   public static final String  DFS_NAMENODE_DECOMMISSION_NODES_PER_INTERVAL_KEY = "dfs.namenode.decommission.nodes.per.interval";

+ 32 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -151,6 +151,7 @@ public class DFSOutputStream extends FSOutputSummer
   private final short blockReplication; // replication factor of file
   private boolean shouldSyncBlock = false; // force blocks to disk upon close
   private CachingStrategy cachingStrategy;
+  private boolean failPacket = false;
   
   private static class Packet {
     private static final long HEART_BEAT_SEQNO = -1L;
@@ -752,6 +753,16 @@ public class DFSOutputStream extends FSOutputSummer
                                     one.seqno + " but received " + seqno);
             }
             isLastPacketInBlock = one.lastPacketInBlock;
+
+            // Fail the packet write for testing in order to force a
+            // pipeline recovery.
+            if (DFSClientFaultInjector.get().failPacket() &&
+                isLastPacketInBlock) {
+              failPacket = true;
+              throw new IOException(
+                    "Failing the last packet for testing.");
+            }
+              
             // update bytesAcked
             block.setNumBytes(one.getLastByteOffsetBlock());
 
@@ -1044,7 +1055,18 @@ public class DFSOutputStream extends FSOutputSummer
         accessToken = lb.getBlockToken();
         
         // set up the pipeline again with the remaining nodes
-        success = createBlockOutputStream(nodes, newGS, isRecovery);
+        if (failPacket) { // for testing
+          success = createBlockOutputStream(nodes, newGS-1, isRecovery);
+          failPacket = false;
+          try {
+            // Give DNs time to send in bad reports. In real situations,
+            // good reports should follow bad ones, if client committed
+            // with those nodes.
+            Thread.sleep(2000);
+          } catch (InterruptedException ie) {}
+        } else {
+          success = createBlockOutputStream(nodes, newGS, isRecovery);
+        }
       }
 
       if (success) {
@@ -1904,7 +1926,9 @@ public class DFSOutputStream extends FSOutputSummer
   // be called during unit tests
   private void completeFile(ExtendedBlock last) throws IOException {
     long localstart = Time.now();
+    long localTimeout = 400;
     boolean fileComplete = false;
+    int retries = dfsClient.getConf().nBlockWriteLocateFollowingRetry;
     while (!fileComplete) {
       fileComplete =
           dfsClient.namenode.complete(src, dfsClient.clientName, last, fileId);
@@ -1920,7 +1944,13 @@ public class DFSOutputStream extends FSOutputSummer
             throw new IOException(msg);
         }
         try {
-          Thread.sleep(400);
+          Thread.sleep(localTimeout);
+          if (retries == 0) {
+            throw new IOException("Unable to close file because the last block"
+                + " does not have enough number of replicas.");
+          }
+          retries--;
+          localTimeout *= 2;
           if (Time.now() - localstart > 5000) {
             DFSClient.LOG.info("Could not complete " + src + " retrying...");
           }

+ 25 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java

@@ -229,6 +229,29 @@ public class BlockInfoUnderConstruction extends BlockInfo {
     return blockRecoveryId;
   }
 
+  /**
+   * Process the recorded replicas. When about to commit or finish the
+   * pipeline recovery sort out bad replicas.
+   * @param genStamp  The final generation stamp for the block.
+   */
+  public void setGenerationStampAndVerifyReplicas(long genStamp) {
+    if (replicas == null)
+      return;
+
+    // Remove the replicas with wrong gen stamp.
+    // The replica list is unchanged.
+    for (ReplicaUnderConstruction r : replicas) {
+      if (genStamp != r.getGenerationStamp()) {
+        r.getExpectedStorageLocation().removeBlock(this);
+        NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica "
+            + "from location: " + r);
+      }
+    }
+
+    // Set the generation stamp for the block.
+    setGenerationStamp(genStamp);
+  }
+
   /**
    * Commit block's length and generation stamp as reported by the client.
    * Set block state to {@link BlockUCState#COMMITTED}.
@@ -301,6 +324,8 @@ public class BlockInfoUnderConstruction extends BlockInfo {
     while (it.hasNext()) {
       ReplicaUnderConstruction r = it.next();
       if(r.getExpectedStorageLocation() == storage) {
+        // Record the gen stamp from the report
+        r.setGenerationStamp(block.getGenerationStamp());
         return;
       } else if (r.getExpectedStorageLocation().getDatanodeDescriptor() ==
           storage.getDatanodeDescriptor()) {

+ 34 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -59,6 +59,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason;
 import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
@@ -1051,7 +1052,8 @@ public class BlockManager {
           + blk + " not found");
       return;
     }
-    markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock, reason), dn, storageID);
+    markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock, reason,
+        Reason.CORRUPTION_REPORTED), dn, storageID);
   }
 
   private void markBlockAsCorrupt(BlockToMarkCorrupt b,
@@ -1074,7 +1076,8 @@ public class BlockManager {
     node.addBlock(storageID, b.stored);
 
     // Add this replica to corruptReplicas Map
-    corruptReplicas.addToCorruptReplicasMap(b.corrupted, node, b.reason);
+    corruptReplicas.addToCorruptReplicasMap(b.corrupted, node, b.reason,
+        b.reasonCode);
     if (countNodes(b.stored).liveReplicas() >= bc.getBlockReplication()) {
       // the block is over-replicated so invalidate the replicas immediately
       invalidateBlock(b, node);
@@ -1574,22 +1577,27 @@ public class BlockManager {
     final BlockInfo stored;
     /** The reason to mark corrupt. */
     final String reason;
-    
-    BlockToMarkCorrupt(BlockInfo corrupted, BlockInfo stored, String reason) {
+    /** The reason code to be stored */
+    final Reason reasonCode;
+
+    BlockToMarkCorrupt(BlockInfo corrupted, BlockInfo stored, String reason,
+        Reason reasonCode) {
       Preconditions.checkNotNull(corrupted, "corrupted is null");
       Preconditions.checkNotNull(stored, "stored is null");
 
       this.corrupted = corrupted;
       this.stored = stored;
       this.reason = reason;
+      this.reasonCode = reasonCode;
     }
 
-    BlockToMarkCorrupt(BlockInfo stored, String reason) {
-      this(stored, stored, reason);
+    BlockToMarkCorrupt(BlockInfo stored, String reason, Reason reasonCode) {
+      this(stored, stored, reason, reasonCode);
     }
 
-    BlockToMarkCorrupt(BlockInfo stored, long gs, String reason) {
-      this(new BlockInfo(stored), stored, reason);
+    BlockToMarkCorrupt(BlockInfo stored, long gs, String reason,
+        Reason reasonCode) {
+      this(new BlockInfo(stored), stored, reason, reasonCode);
       //the corrupted block in datanode has a different generation stamp
       corrupted.setGenerationStamp(gs);
     }
@@ -1946,9 +1954,11 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       return storedBlock;
     }
 
-    //add replica if appropriate
+    // Add replica if appropriate. If the replica was previously corrupt
+    // but now okay, it might need to be updated.
     if (reportedState == ReplicaState.FINALIZED
-        && storedBlock.findDatanode(dn) < 0) {
+        && (storedBlock.findDatanode(dn) < 0
+        || corruptReplicas.isReplicaCorrupt(storedBlock, dn))) {
       toAdd.add(storedBlock);
     }
     return storedBlock;
@@ -2039,12 +2049,13 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
           return new BlockToMarkCorrupt(storedBlock, reportedGS,
               "block is " + ucState + " and reported genstamp " + reportedGS
               + " does not match genstamp in block map "
-              + storedBlock.getGenerationStamp());
+              + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
         } else if (storedBlock.getNumBytes() != reported.getNumBytes()) {
           return new BlockToMarkCorrupt(storedBlock,
               "block is " + ucState + " and reported length " +
               reported.getNumBytes() + " does not match " +
-              "length in block map " + storedBlock.getNumBytes());
+              "length in block map " + storedBlock.getNumBytes(),
+              Reason.SIZE_MISMATCH);
         } else {
           return null; // not corrupt
         }
@@ -2060,7 +2071,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
         return new BlockToMarkCorrupt(storedBlock, reportedGS,
             "reported " + reportedState + " replica with genstamp " + reportedGS
             + " does not match COMPLETE block's genstamp in block map "
-            + storedBlock.getGenerationStamp());
+            + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
       } else { // COMPLETE block, same genstamp
         if (reportedState == ReplicaState.RBW) {
           // If it's a RBW report for a COMPLETE block, it may just be that
@@ -2073,7 +2084,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
           return null;
         } else {
           return new BlockToMarkCorrupt(storedBlock,
-              "reported replica has invalid state " + reportedState);
+              "reported replica has invalid state " + reportedState,
+              Reason.INVALID_STATE);
         }
       }
     case RUR:       // should not be reported
@@ -2084,7 +2096,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       " on " + dn + " size " + storedBlock.getNumBytes();
       // log here at WARN level since this is really a broken HDFS invariant
       LOG.warn(msg);
-      return new BlockToMarkCorrupt(storedBlock, msg);
+      return new BlockToMarkCorrupt(storedBlock, msg, Reason.INVALID_STATE);
     }
   }
 
@@ -2201,6 +2213,11 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
         logAddStoredBlock(storedBlock, node);
       }
     } else {
+      // if the same block is added again and the replica was corrupt
+      // previously because of a wrong gen stamp, remove it from the
+      // corrupt block list.
+      corruptReplicas.removeFromCorruptReplicasMap(block, node,
+          Reason.GENSTAMP_MISMATCH);
       curReplicaDelta = 0;
       blockLog.warn("BLOCK* addStoredBlock: "
           + "Redundant addStoredBlock request received for " + storedBlock
@@ -2297,7 +2314,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     DatanodeDescriptor[] nodesCopy = nodes.toArray(new DatanodeDescriptor[0]);
     for (DatanodeDescriptor node : nodesCopy) {
       try {
-        if (!invalidateBlock(new BlockToMarkCorrupt(blk, null), node)) {
+        if (!invalidateBlock(new BlockToMarkCorrupt(blk, null,
+              Reason.ANY), node)) {
           removedFromBlocksMap = false;
         }
       } catch (IOException e) {

+ 9 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java

@@ -204,6 +204,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
     namesystem.writeLock();
     try {
       rescanCachedBlockMap();
+      blockManager.getDatanodeManager().resetLastCachingDirectiveSentTime();
     } finally {
       namesystem.writeUnlock();
     }
@@ -316,17 +317,21 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       int numCached = cached.size();
       if (numCached >= neededCached) {
         // If we have enough replicas, drop all pending cached.
-        for (DatanodeDescriptor datanode : pendingCached) {
+        for (Iterator<DatanodeDescriptor> iter = pendingCached.iterator();
+            iter.hasNext(); ) {
+          DatanodeDescriptor datanode = iter.next();
           datanode.getPendingCached().remove(cblock);
+          iter.remove();
         }
-        pendingCached.clear();
       }
       if (numCached < neededCached) {
         // If we don't have enough replicas, drop all pending uncached.
-        for (DatanodeDescriptor datanode : pendingUncached) {
+        for (Iterator<DatanodeDescriptor> iter = pendingUncached.iterator();
+            iter.hasNext(); ) {
+          DatanodeDescriptor datanode = iter.next();
           datanode.getPendingUncached().remove(cblock);
+          iter.remove();
         }
-        pendingUncached.clear();
       }
       int neededUncached = numCached -
           (pendingUncached.size() + neededCached);

+ 50 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java

@@ -36,8 +36,18 @@ import java.util.*;
 @InterfaceAudience.Private
 public class CorruptReplicasMap{
 
-  private SortedMap<Block, Collection<DatanodeDescriptor>> corruptReplicasMap =
-    new TreeMap<Block, Collection<DatanodeDescriptor>>();
+  /** The corruption reason code */
+  public static enum Reason {
+    NONE,                // not specified.
+    ANY,                 // wildcard reason
+    GENSTAMP_MISMATCH,   // mismatch in generation stamps
+    SIZE_MISMATCH,       // mismatch in sizes
+    INVALID_STATE,       // invalid state
+    CORRUPTION_REPORTED  // client or datanode reported the corruption
+  }
+
+  private SortedMap<Block, Map<DatanodeDescriptor, Reason>> corruptReplicasMap =
+    new TreeMap<Block, Map<DatanodeDescriptor, Reason>>();
   
   /**
    * Mark the block belonging to datanode as corrupt.
@@ -48,9 +58,22 @@ public class CorruptReplicasMap{
    */
   public void addToCorruptReplicasMap(Block blk, DatanodeDescriptor dn,
       String reason) {
-    Collection<DatanodeDescriptor> nodes = getNodes(blk);
+    addToCorruptReplicasMap(blk, dn, reason, Reason.NONE);
+  }
+
+  /**
+   * Mark the block belonging to datanode as corrupt.
+   *
+   * @param blk Block to be added to CorruptReplicasMap
+   * @param dn DatanodeDescriptor which holds the corrupt replica
+   * @param reason a textual reason (for logging purposes)
+   * @param reasonCode the enum representation of the reason
+   */
+  public void addToCorruptReplicasMap(Block blk, DatanodeDescriptor dn,
+      String reason, Reason reasonCode) {
+    Map <DatanodeDescriptor, Reason> nodes = corruptReplicasMap.get(blk);
     if (nodes == null) {
-      nodes = new TreeSet<DatanodeDescriptor>();
+      nodes = new HashMap<DatanodeDescriptor, Reason>();
       corruptReplicasMap.put(blk, nodes);
     }
     
@@ -61,8 +84,7 @@ public class CorruptReplicasMap{
       reasonText = "";
     }
     
-    if (!nodes.contains(dn)) {
-      nodes.add(dn);
+    if (!nodes.keySet().contains(dn)) {
       NameNode.blockStateChangeLog.info("BLOCK NameSystem.addToCorruptReplicasMap: "+
                                    blk.getBlockName() +
                                    " added as corrupt on " + dn +
@@ -76,6 +98,8 @@ public class CorruptReplicasMap{
                                    " by " + Server.getRemoteIp() +
                                    reasonText);
     }
+    // Add the node or update the reason.
+    nodes.put(dn, reasonCode);
   }
 
   /**
@@ -97,10 +121,24 @@ public class CorruptReplicasMap{
              false if the replica is not in the map
    */ 
   boolean removeFromCorruptReplicasMap(Block blk, DatanodeDescriptor datanode) {
-    Collection<DatanodeDescriptor> datanodes = corruptReplicasMap.get(blk);
+    return removeFromCorruptReplicasMap(blk, datanode, Reason.ANY);
+  }
+
+  boolean removeFromCorruptReplicasMap(Block blk, DatanodeDescriptor datanode,
+      Reason reason) {
+    Map <DatanodeDescriptor, Reason> datanodes = corruptReplicasMap.get(blk);
+    boolean removed = false;
     if (datanodes==null)
       return false;
-    if (datanodes.remove(datanode)) { // remove the replicas
+
+    // if reasons can be compared but don't match, return false.
+    Reason storedReason = datanodes.get(datanode);
+    if (reason != Reason.ANY && storedReason != null &&
+        reason != storedReason) {
+      return false;
+    }
+
+    if (datanodes.remove(datanode) != null) { // remove the replicas
       if (datanodes.isEmpty()) {
         // remove the block if there is no more corrupted replicas
         corruptReplicasMap.remove(blk);
@@ -118,7 +156,10 @@ public class CorruptReplicasMap{
    * @return collection of nodes. Null if does not exists
    */
   Collection<DatanodeDescriptor> getNodes(Block blk) {
-    return corruptReplicasMap.get(blk);
+    Map <DatanodeDescriptor, Reason> nodes = corruptReplicasMap.get(blk);
+    if (nodes == null)
+      return null;
+    return nodes.keySet();
   }
 
   /**

+ 31 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -170,6 +170,21 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return pendingUncached;
   }
 
+  /**
+   * The time when the last batch of caching directives was sent, in
+   * monotonic milliseconds.
+   */
+  private long lastCachingDirectiveSentTimeMs;
+
+  /**
+   * Head of the list of blocks on the datanode
+   */
+  private volatile BlockInfo blockList = null;
+  /**
+   * Number of blocks on the datanode
+   */
+  private int numBlocks = 0;
+
   // isAlive == heartbeats.contains(this)
   // This is an optimization, because contains takes O(n) time on Arraylist
   public boolean isAlive = false;
@@ -661,5 +676,21 @@ public class DatanodeDescriptor extends DatanodeInfo {
       return storage;
     }
   }
+
+  /**
+   * @return   The time at which we last sent caching directives to this 
+   *           DataNode, in monotonic milliseconds.
+   */
+  public long getLastCachingDirectiveSentTimeMs() {
+    return this.lastCachingDirectiveSentTimeMs;
+  }
+
+  /**
+   * @param time  The time at which we last sent caching directives to this 
+   *              DataNode, in monotonic milliseconds.
+   */
+  public void setLastCachingDirectiveSentTimeMs(long time) {
+    this.lastCachingDirectiveSentTimeMs = time;
+  }
 }
 

+ 51 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -149,7 +149,7 @@ public class DatanodeManager {
    * Whether we should tell datanodes what to cache in replies to
    * heartbeat messages.
    */
-  private boolean sendCachingCommands = false;
+  private boolean shouldSendCachingCommands = false;
 
   /**
    * The number of datanodes for each software version. This list should change
@@ -159,6 +159,16 @@ public class DatanodeManager {
   private HashMap<String, Integer> datanodesSoftwareVersions =
     new HashMap<String, Integer>(4, 0.75f);
   
+  /**
+   * The minimum time between resending caching directives to Datanodes,
+   * in milliseconds.
+   *
+   * Note that when a rescan happens, we will send the new directives
+   * as soon as possible.  This timeout only applies to resending 
+   * directives that we've already sent.
+   */
+  private final long timeBetweenResendingCachingDirectivesMs;
+  
   DatanodeManager(final BlockManager blockManager, final Namesystem namesystem,
       final Configuration conf) throws IOException {
     this.namesystem = namesystem;
@@ -241,6 +251,9 @@ public class DatanodeManager {
         DFSConfigKeys.DFS_NAMENODE_USE_STALE_DATANODE_FOR_WRITE_RATIO_KEY +
         " = '" + ratioUseStaleDataNodesForWrite + "' is invalid. " +
         "It should be a positive non-zero float value, not greater than 1.0f.");
+    this.timeBetweenResendingCachingDirectivesMs = conf.getLong(
+        DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_RETRY_INTERVAL_MS,
+        DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_RETRY_INTERVAL_MS_DEFAULT);
   }
 
   private static long getStaleIntervalFromConf(Configuration conf,
@@ -1297,17 +1310,28 @@ public class DatanodeManager {
           cmds.add(new BlockCommand(DatanodeProtocol.DNA_INVALIDATE,
               blockPoolId, blks));
         }
-        DatanodeCommand pendingCacheCommand =
-            getCacheCommand(nodeinfo.getPendingCached(), nodeinfo,
-              DatanodeProtocol.DNA_CACHE, blockPoolId);
-        if (pendingCacheCommand != null) {
-          cmds.add(pendingCacheCommand);
-        }
-        DatanodeCommand pendingUncacheCommand =
-            getCacheCommand(nodeinfo.getPendingUncached(), nodeinfo,
-              DatanodeProtocol.DNA_UNCACHE, blockPoolId);
-        if (pendingUncacheCommand != null) {
-          cmds.add(pendingUncacheCommand);
+        boolean sendingCachingCommands = false;
+        long nowMs = Time.monotonicNow();
+        if (shouldSendCachingCommands && 
+            ((nowMs - nodeinfo.getLastCachingDirectiveSentTimeMs()) >=
+                timeBetweenResendingCachingDirectivesMs)) {
+          DatanodeCommand pendingCacheCommand =
+              getCacheCommand(nodeinfo.getPendingCached(), nodeinfo,
+                DatanodeProtocol.DNA_CACHE, blockPoolId);
+          if (pendingCacheCommand != null) {
+            cmds.add(pendingCacheCommand);
+            sendingCachingCommands = true;
+          }
+          DatanodeCommand pendingUncacheCommand =
+              getCacheCommand(nodeinfo.getPendingUncached(), nodeinfo,
+                DatanodeProtocol.DNA_UNCACHE, blockPoolId);
+          if (pendingUncacheCommand != null) {
+            cmds.add(pendingUncacheCommand);
+            sendingCachingCommands = true;
+          }
+          if (sendingCachingCommands) {
+            nodeinfo.setLastCachingDirectiveSentTimeMs(nowMs);
+          }
         }
 
         blockManager.addKeyUpdateCommand(cmds, nodeinfo);
@@ -1345,19 +1369,13 @@ public class DatanodeManager {
     if (length == 0) {
       return null;
     }
-    // Read and clear the existing cache commands.
+    // Read the existing cache commands.
     long[] blockIds = new long[length];
     int i = 0;
     for (Iterator<CachedBlock> iter = list.iterator();
             iter.hasNext(); ) {
       CachedBlock cachedBlock = iter.next();
       blockIds[i++] = cachedBlock.getBlockId();
-      iter.remove();
-    }
-    if (!sendCachingCommands) {
-      // Do not send caching commands unless the FSNamesystem told us we
-      // should.
-      return null;
     }
     return new BlockIdCommand(action, poolId, blockIds);
   }
@@ -1408,13 +1426,25 @@ public class DatanodeManager {
     }
   }
 
+  /**
+   * Reset the lastCachingDirectiveSentTimeMs field of all the DataNodes we
+   * know about.
+   */
+  public void resetLastCachingDirectiveSentTime() {
+    synchronized (datanodeMap) {
+      for (DatanodeDescriptor dn : datanodeMap.values()) {
+        dn.setLastCachingDirectiveSentTimeMs(0L);
+      }
+    }
+  }
+
   @Override
   public String toString() {
     return getClass().getSimpleName() + ": " + host2DatanodeMap;
   }
 
-  public void setSendCachingCommands(boolean sendCachingCommands) {
-    this.sendCachingCommands = sendCachingCommands;
+  public void setShouldSendCachingCommands(boolean shouldSendCachingCommands) {
+    this.shouldSendCachingCommands = shouldSendCachingCommands;
   }
 }
 

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java

@@ -289,6 +289,10 @@ public class FsDatasetCache {
     mappableBlockMap.put(key, new Value(null, State.CACHING));
     volumeExecutor.execute(
         new CachingTask(key, blockFileName, length, genstamp));
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Initiating caching for Block with id " + blockId +
+          ", pool " + bpid);
+    }
   }
 
   synchronized void uncacheBlock(String bpid, long blockId) {
@@ -427,6 +431,10 @@ public class FsDatasetCache {
             mappableBlock.close();
           }
           numBlocksFailedToCache.incrementAndGet();
+
+          synchronized (FsDatasetCache.this) {
+            mappableBlockMap.remove(key);
+          }
         }
       }
     }

+ 1 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/MappableBlock.java

@@ -44,20 +44,6 @@ import com.google.common.base.Preconditions;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class MappableBlock implements Closeable {
-  public static interface Mlocker {
-    void mlock(MappedByteBuffer mmap, long length) throws IOException;
-  }
-  
-  private static class PosixMlocker implements Mlocker {
-    public void mlock(MappedByteBuffer mmap, long length)
-        throws IOException {
-      NativeIO.POSIX.mlock(mmap, length);
-    }
-  }
-
-  @VisibleForTesting
-  public static Mlocker mlocker = new PosixMlocker();
-
   private MappedByteBuffer mmap;
   private final long length;
 
@@ -96,7 +82,7 @@ public class MappableBlock implements Closeable {
         throw new IOException("Block InputStream has no FileChannel.");
       }
       mmap = blockChannel.map(MapMode.READ_ONLY, 0, length);
-      mlocker.mlock(mmap, length);
+      NativeIO.POSIX.cacheManipulator.mlock(blockFileName, mmap, length);
       verifyChecksum(length, metaIn, blockChannel, blockFileName);
       mappableBlock = new MappableBlock(mmap, length);
     } finally {

+ 119 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryComputationContext.java

@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ContentSummaryComputationContext {
+  private FSDirectory dir = null;
+  private FSNamesystem fsn = null;
+  private Content.Counts counts = null;
+  private long nextCountLimit = 0;
+  private long limitPerRun = 0;
+  private long yieldCount = 0;
+
+  /**
+   * Constructor
+   *
+   * @param dir The FSDirectory instance
+   * @param fsn The FSNamesystem instance
+   * @param limitPerRun allowed number of operations in one
+   *        locking period. 0 or a negative number means
+   *        no limit (i.e. no yielding)
+   */
+  public ContentSummaryComputationContext(FSDirectory dir,
+      FSNamesystem fsn, long limitPerRun) {
+    this.dir = dir;
+    this.fsn = fsn;
+    this.limitPerRun = limitPerRun;
+    this.nextCountLimit = limitPerRun;
+    this.counts = Content.Counts.newInstance();
+  }
+
+  /** Constructor for blocking computation. */
+  public ContentSummaryComputationContext() {
+    this(null, null, 0);
+  }
+
+  /** Return current yield count */
+  public long getYieldCount() {
+    return yieldCount;
+  }
+
+  /**
+   * Relinquish locks held during computation for a short while
+   * and reacquire them. This will give other threads a chance
+   * to acquire the contended locks and run.
+   *
+   * @return true if locks were released and reacquired.
+   */
+  public boolean yield() {
+    // Are we set up to do this?
+    if (limitPerRun <= 0 || dir == null || fsn == null) {
+      return false;
+    }
+
+    // Have we reached the limit?
+    long currentCount = counts.get(Content.FILE) +
+        counts.get(Content.SYMLINK) +
+        counts.get(Content.DIRECTORY) +
+        counts.get(Content.SNAPSHOTTABLE_DIRECTORY);
+    if (currentCount <= nextCountLimit) {
+      return false;
+    }
+
+    // Update the next limit
+    nextCountLimit = currentCount + limitPerRun;
+
+    boolean hadDirReadLock = dir.hasReadLock();
+    boolean hadDirWriteLock = dir.hasWriteLock();
+    boolean hadFsnReadLock = fsn.hasReadLock();
+    boolean hadFsnWriteLock = fsn.hasWriteLock();
+
+    // sanity check.
+    if (!hadDirReadLock || !hadFsnReadLock || hadDirWriteLock ||
+        hadFsnWriteLock || dir.getReadHoldCount() != 1 ||
+        fsn.getReadHoldCount() != 1) {
+      // cannot relinquish
+      return false;
+    }
+
+    // unlock
+    dir.readUnlock();
+    fsn.readUnlock();
+
+    try {
+      Thread.sleep(1);
+    } catch (InterruptedException ie) {
+    } finally {
+      // reacquire
+      fsn.readLock();
+      dir.readLock();
+    }
+    yieldCount++;
+    return true;
+  }
+
+  /** Get the content counts */
+  public Content.Counts getCounts() {
+    return counts;
+  }
+}

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

@@ -114,7 +114,9 @@ public class FSDirectory implements Closeable {
   private final int maxComponentLength;
   private final int maxDirItems;
   private final int lsLimit;  // max list limit
+  private final int contentCountLimit; // max content summary counts per run
   private final INodeMap inodeMap; // Synchronized by dirLock
+  private long yieldCount = 0; // keep track of lock yield count.
 
   // lock to protect the directory and BlockMap
   private ReentrantReadWriteLock dirLock;
@@ -145,6 +147,14 @@ public class FSDirectory implements Closeable {
     return this.dirLock.getReadHoldCount() > 0;
   }
 
+  public int getReadHoldCount() {
+    return this.dirLock.getReadHoldCount();
+  }
+
+  public int getWriteHoldCount() {
+    return this.dirLock.getWriteHoldCount();
+  }
+
   /**
    * Caches frequently used file names used in {@link INode} to reuse 
    * byte[] objects and reduce heap usage.
@@ -161,6 +171,10 @@ public class FSDirectory implements Closeable {
         DFSConfigKeys.DFS_LIST_LIMIT, DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT);
     this.lsLimit = configuredLimit>0 ?
         configuredLimit : DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT;
+
+    this.contentCountLimit = conf.getInt(
+        DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_KEY,
+        DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_DEFAULT);
     
     // filesystem limits
     this.maxComponentLength = conf.getInt(
@@ -2296,13 +2310,26 @@ public class FSDirectory implements Closeable {
         throw new FileNotFoundException("File does not exist: " + srcs);
       }
       else {
-        return targetNode.computeContentSummary();
+        // Make it relinquish locks everytime contentCountLimit entries are
+        // processed. 0 means disabled. I.e. blocking for the entire duration.
+        ContentSummaryComputationContext cscc =
+
+            new ContentSummaryComputationContext(this, getFSNamesystem(),
+            contentCountLimit);
+        ContentSummary cs = targetNode.computeAndConvertContentSummary(cscc);
+        yieldCount += cscc.getYieldCount();
+        return cs;
       }
     } finally {
       readUnlock();
     }
   }
 
+  @VisibleForTesting
+  public long getYieldCount() {
+    return yieldCount;
+  }
+
   public INodeMap getINodeMap() {
     return inodeMap;
   }

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

@@ -593,7 +593,7 @@ public class FSEditLogLoader {
       fsNamesys.getSnapshotManager().deleteSnapshot(
           deleteSnapshotOp.snapshotRoot, deleteSnapshotOp.snapshotName,
           collectedBlocks, removedINodes);
-      fsNamesys.removeBlocks(collectedBlocks);
+      fsNamesys.removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
       collectedBlocks.clear();
       fsNamesys.dir.removeFromInodeMap(removedINodes);
       removedINodes.clear();

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

@@ -1009,7 +1009,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       nnEditLogRoller.start();
 
       cacheManager.activate();
-      blockManager.getDatanodeManager().setSendCachingCommands(true);
+      blockManager.getDatanodeManager().setShouldSendCachingCommands(true);
     } finally {
       writeUnlock();
       startingActiveService = false;
@@ -1060,7 +1060,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         dir.fsImage.updateLastAppliedTxIdFromWritten();
       }
       cacheManager.deactivate();
-      blockManager.getDatanodeManager().setSendCachingCommands(false);
+      blockManager.getDatanodeManager().setShouldSendCachingCommands(false);
     } finally {
       writeUnlock();
     }
@@ -1297,6 +1297,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return hasReadLock() || hasWriteLock();
   }
 
+  public int getReadHoldCount() {
+    return this.fsLock.getReadHoldCount();
+  }
+
+  public int getWriteHoldCount() {
+    return this.fsLock.getWriteHoldCount();
+  }
+
   NamespaceInfo getNamespaceInfo() {
     readLock();
     try {
@@ -3305,6 +3313,18 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       return;
     }
     
+    removeBlocksAndUpdateSafemodeTotal(blocks);
+  }
+
+  /**
+   * Removes the blocks from blocksmap and updates the safemode blocks total
+   * 
+   * @param blocks
+   *          An instance of {@link BlocksMapUpdateInfo} which contains a list
+   *          of blocks that need to be removed from blocksMap
+   */
+  void removeBlocksAndUpdateSafemodeTotal(BlocksMapUpdateInfo blocks) {
+    assert hasWriteLock();
     // In the case that we are a Standby tailing edits from the
     // active while in safe-mode, we need to track the total number
     // of blocks and safe blocks in the system.
@@ -3325,9 +3345,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
     if (trackBlockCounts) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Adjusting safe-mode totals for deletion of " + src + ":" +
-            "decreasing safeBlocks by " + numRemovedSafe +
-            ", totalBlocks by " + numRemovedComplete);
+        LOG.debug("Adjusting safe-mode totals for deletion."
+            + "decreasing safeBlocks by " + numRemovedSafe
+            + ", totalBlocks by " + numRemovedComplete);
       }
       adjustSafeModeBlockTotals(-numRemovedSafe, -numRemovedComplete);
     }
@@ -5883,8 +5903,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
 
     // Update old block with the new generation stamp and new length
-    blockinfo.setGenerationStamp(newBlock.getGenerationStamp());
     blockinfo.setNumBytes(newBlock.getNumBytes());
+    blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp());
 
     // find the DatanodeDescriptor objects
     final DatanodeStorageInfo[] storages = blockManager.getDatanodeManager()
@@ -6953,6 +6973,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       return; // Return previous response
     }
     boolean success = false;
+    BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -6961,7 +6982,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         checkOwner(pc, snapshotRoot);
       }
 
-      BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
       List<INode> removedINodes = new ChunkedArrayList<INode>();
       dir.writeLock();
       try {
@@ -6972,8 +6992,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         dir.writeUnlock();
       }
       removedINodes.clear();
-      this.removeBlocks(collectedBlocks);
-      collectedBlocks.clear();
       getEditLog().logDeleteSnapshot(snapshotRoot, snapshotName,
           cacheEntry != null);
       success = true;
@@ -6982,7 +7000,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       RetryCache.setState(cacheEntry, success);
     }
     getEditLog().logSync();
-    
+
+    removeBlocks(collectedBlocks);
+    collectedBlocks.clear();
+
     if (auditLog.isInfoEnabled() && isExternalInvocation()) {
       String rootPath = Snapshot.getSnapshotPath(snapshotRoot, snapshotName);
       logAuditEvent(true, "deleteSnapshot", rootPath, null, null);

+ 16 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -371,10 +371,18 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   public abstract void destroyAndCollectBlocks(
       BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes);
 
-  /** Compute {@link ContentSummary}. */
+  /** Compute {@link ContentSummary}. Blocking call */
   public final ContentSummary computeContentSummary() {
-    final Content.Counts counts = computeContentSummary(
-        Content.Counts.newInstance());
+    return computeAndConvertContentSummary(
+        new ContentSummaryComputationContext());
+  }
+
+  /**
+   * Compute {@link ContentSummary}. 
+   */
+  public final ContentSummary computeAndConvertContentSummary(
+      ContentSummaryComputationContext summary) {
+    Content.Counts counts = computeContentSummary(summary).getCounts();
     return new ContentSummary(counts.get(Content.LENGTH),
         counts.get(Content.FILE) + counts.get(Content.SYMLINK),
         counts.get(Content.DIRECTORY), getNsQuota(),
@@ -384,10 +392,12 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   /**
    * Count subtree content summary with a {@link Content.Counts}.
    *
-   * @param counts The subtree counts for returning.
-   * @return The same objects as the counts parameter.
+   * @param summary the context object holding counts for the subtree.
+   * @return The same objects as summary.
    */
-  public abstract Content.Counts computeContentSummary(Content.Counts counts);
+  public abstract ContentSummaryComputationContext computeContentSummary(
+      ContentSummaryComputationContext summary);
+
   
   /**
    * Check and add namespace/diskspace consumed to itself and the ancestors.

+ 38 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -466,12 +466,45 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
 
   @Override
-  public Content.Counts computeContentSummary(final Content.Counts counts) {
-    for (INode child : getChildrenList(null)) {
-      child.computeContentSummary(counts);
+  public ContentSummaryComputationContext computeContentSummary(
+      ContentSummaryComputationContext summary) {
+    ReadOnlyList<INode> childrenList = getChildrenList(null);
+    // Explicit traversing is done to enable repositioning after relinquishing
+    // and reacquiring locks.
+    for (int i = 0;  i < childrenList.size(); i++) {
+      INode child = childrenList.get(i);
+      byte[] childName = child.getLocalNameBytes();
+
+      long lastYieldCount = summary.getYieldCount();
+      child.computeContentSummary(summary);
+
+      // Check whether the computation was paused in the subtree.
+      // The counts may be off, but traversing the rest of children
+      // should be made safe.
+      if (lastYieldCount == summary.getYieldCount()) {
+        continue;
+      }
+
+      // The locks were released and reacquired. Check parent first.
+      if (getParent() == null) {
+        // Stop further counting and return whatever we have so far.
+        break;
+      }
+
+      // Obtain the children list again since it may have been modified.
+      childrenList = getChildrenList(null);
+      // Reposition in case the children list is changed. Decrement by 1
+      // since it will be incremented when loops.
+      i = nextChild(childrenList, childName) - 1;
     }
-    counts.add(Content.DIRECTORY, 1);
-    return counts;
+
+    // Increment the directory count for this directory.
+    summary.getCounts().add(Content.DIRECTORY, 1);
+
+    // Relinquish and reacquire locks if necessary.
+    summary.yield();
+
+    return summary;
   }
 
   /**

+ 10 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java

@@ -107,12 +107,16 @@ public class INodeDirectoryWithQuota extends INodeDirectory {
   }
 
   @Override
-  public Content.Counts computeContentSummary(
-      final Content.Counts counts) {
-    final long original = counts.get(Content.DISKSPACE);
-    super.computeContentSummary(counts);
-    checkDiskspace(counts.get(Content.DISKSPACE) - original);
-    return counts;
+  public ContentSummaryComputationContext computeContentSummary(
+      final ContentSummaryComputationContext summary) {
+    final long original = summary.getCounts().get(Content.DISKSPACE);
+    long oldYieldCount = summary.getYieldCount();
+    super.computeContentSummary(summary);
+    // Check only when the content has not changed in the middle.
+    if (oldYieldCount == summary.getYieldCount()) {
+      checkDiskspace(summary.getCounts().get(Content.DISKSPACE) - original);
+    }
+    return summary;
   }
   
   private void checkDiskspace(final long computed) {

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

@@ -342,11 +342,11 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   @Override
-  public final Content.Counts computeContentSummary(
-      final Content.Counts counts) {
-    computeContentSummary4Snapshot(counts);
-    computeContentSummary4Current(counts);
-    return counts;
+  public final ContentSummaryComputationContext  computeContentSummary(
+      final ContentSummaryComputationContext summary) {
+    computeContentSummary4Snapshot(summary.getCounts());
+    computeContentSummary4Current(summary.getCounts());
+    return summary;
   }
 
   private void computeContentSummary4Snapshot(final Content.Counts counts) {

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

@@ -107,7 +107,8 @@ public class INodeMap {
       }
       
       @Override
-      public Content.Counts computeContentSummary(Content.Counts counts) {
+      public ContentSummaryComputationContext computeContentSummary(
+          ContentSummaryComputationContext summary) {
         return null;
       }
       

+ 8 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java

@@ -278,8 +278,9 @@ public abstract class INodeReference extends INode {
   }
 
   @Override
-  public Content.Counts computeContentSummary(Content.Counts counts) {
-    return referred.computeContentSummary(counts);
+  public ContentSummaryComputationContext computeContentSummary(
+      ContentSummaryComputationContext summary) {
+    return referred.computeContentSummary(summary);
   }
 
   @Override
@@ -444,12 +445,13 @@ public abstract class INodeReference extends INode {
     }
     
     @Override
-    public final Content.Counts computeContentSummary(Content.Counts counts) {
+    public final ContentSummaryComputationContext computeContentSummary(
+        ContentSummaryComputationContext summary) {
       //only count diskspace for WithName
       final Quota.Counts q = Quota.Counts.newInstance();
       computeQuotaUsage(q, false, lastSnapshotId);
-      counts.add(Content.DISKSPACE, q.get(Quota.DISKSPACE));
-      return counts;
+      summary.getCounts().add(Content.DISKSPACE, q.get(Quota.DISKSPACE));
+      return summary;
     }
 
     @Override
@@ -688,4 +690,4 @@ public abstract class INodeReference extends INode {
       }
     }
   }
-}
+}

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

@@ -98,9 +98,10 @@ public class INodeSymlink extends INodeWithAdditionalFields {
   }
 
   @Override
-  public Content.Counts computeContentSummary(final Content.Counts counts) {
-    counts.add(Content.SYMLINK, 1);
-    return counts;
+  public ContentSummaryComputationContext computeContentSummary(
+      final ContentSummaryComputationContext summary) {
+    summary.getCounts().add(Content.SYMLINK, 1);
+    return summary;
   }
 
   @Override

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

@@ -38,6 +38,7 @@ 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.server.namenode.Content;
+import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
@@ -342,11 +343,12 @@ public class INodeDirectorySnapshottable extends INodeDirectoryWithSnapshot {
   }
   
   @Override
-  public Content.Counts computeContentSummary(final Content.Counts counts) {
-    super.computeContentSummary(counts);
-    counts.add(Content.SNAPSHOT, snapshotsByNames.size());
-    counts.add(Content.SNAPSHOTTABLE_DIRECTORY, 1);
-    return counts;
+  public ContentSummaryComputationContext computeContentSummary(
+      final ContentSummaryComputationContext summary) {
+    super.computeContentSummary(summary);
+    summary.getCounts().add(Content.SNAPSHOT, snapshotsByNames.size());
+    summary.getCounts().add(Content.SNAPSHOTTABLE_DIRECTORY, 1);
+    return summary;
   }
 
   /**

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

@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.server.namenode.Content;
+import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
@@ -883,18 +884,27 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
   }
 
   @Override
-  public Content.Counts computeContentSummary(final Content.Counts counts) {
-    super.computeContentSummary(counts);
-    computeContentSummary4Snapshot(counts);
-    return counts;
+  public ContentSummaryComputationContext computeContentSummary(
+      final ContentSummaryComputationContext summary) {
+    // Snapshot summary calc won't be relinquishing locks in the middle.
+    // Do this first and handover to parent.
+    computeContentSummary4Snapshot(summary.getCounts());
+    super.computeContentSummary(summary);
+    return summary;
   }
 
   private void computeContentSummary4Snapshot(final Content.Counts counts) {
+    // Create a new blank summary context for blocking processing of subtree.
+    ContentSummaryComputationContext summary = 
+        new ContentSummaryComputationContext();
     for(DirectoryDiff d : diffs) {
       for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
-        deleted.computeContentSummary(counts);
+        deleted.computeContentSummary(summary);
       }
     }
+    // Add the counts from deleted trees.
+    counts.add(summary.getCounts());
+    // Add the deleted directory count.
     counts.add(Content.DIRECTORY, diffs.asList().size());
   }
   

+ 83 - 78
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java

@@ -26,8 +26,8 @@ import java.io.InputStreamReader;
 import java.io.PrintStream;
 import java.net.HttpURLConnection;
 import java.net.InetSocketAddress;
+import java.net.URI;
 import java.net.URL;
-import java.net.URLConnection;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.Date;
@@ -47,11 +47,14 @@ import org.apache.hadoop.hdfs.server.namenode.CancelDelegationTokenServlet;
 import org.apache.hadoop.hdfs.server.namenode.GetDelegationTokenServlet;
 import org.apache.hadoop.hdfs.server.namenode.RenewDelegationTokenServlet;
 import org.apache.hadoop.hdfs.web.HftpFileSystem;
+import org.apache.hadoop.hdfs.web.HsftpFileSystem;
+import org.apache.hadoop.hdfs.web.URLConnectionFactory;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.GenericOptionsParser;
@@ -142,11 +145,11 @@ public class DelegationTokenFetcher {
     // default to using the local file system
     FileSystem local = FileSystem.getLocal(conf);
     final Path tokenFile = new Path(local.getWorkingDirectory(), remaining[0]);
+    final URLConnectionFactory connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
 
     // Login the current user
     UserGroupInformation.getCurrentUser().doAs(
         new PrivilegedExceptionAction<Object>() {
-          @SuppressWarnings("unchecked")
           @Override
           public Object run() throws Exception {
 
@@ -182,7 +185,8 @@ public class DelegationTokenFetcher {
             } else {
               // otherwise we are fetching
               if (webUrl != null) {
-                Credentials creds = getDTfromRemote(webUrl, renewer);
+                Credentials creds = getDTfromRemote(connectionFactory, new URI(webUrl),
+                    renewer);
                 creds.writeTokenStorageFile(tokenFile, conf);
                 for (Token<?> token : creds.getAllTokens()) {
                   if(LOG.isDebugEnabled()) {	
@@ -208,32 +212,31 @@ public class DelegationTokenFetcher {
         });
   }
   
-  static public Credentials getDTfromRemote(String nnAddr, 
-      String renewer) throws IOException {
+  static public Credentials getDTfromRemote(URLConnectionFactory factory,
+      URI nnUri, String renewer) throws IOException {
+    StringBuilder buf = new StringBuilder(nnUri.toString())
+        .append(GetDelegationTokenServlet.PATH_SPEC);
+    if (renewer != null) {
+      buf.append("?").append(GetDelegationTokenServlet.RENEWER).append("=")
+          .append(renewer);
+    }
+
+    HttpURLConnection conn = null;
     DataInputStream dis = null;
-    InetSocketAddress serviceAddr = NetUtils.createSocketAddr(nnAddr);
-    
+    InetSocketAddress serviceAddr = NetUtils.createSocketAddr(nnUri
+        .getAuthority());
+
     try {
-      StringBuffer url = new StringBuffer();
-      if (renewer != null) {
-        url.append(nnAddr).append(GetDelegationTokenServlet.PATH_SPEC)
-           .append("?").append(GetDelegationTokenServlet.RENEWER).append("=")
-           .append(renewer);
-      } else {
-        url.append(nnAddr).append(GetDelegationTokenServlet.PATH_SPEC);
-      }
-      
       if(LOG.isDebugEnabled()) {
-        LOG.debug("Retrieving token from: " + url);
+        LOG.debug("Retrieving token from: " + buf);
       }
-      
-      URL remoteURL = new URL(url.toString());
-      URLConnection connection = SecurityUtil.openSecureHttpConnection(remoteURL);
-      InputStream in = connection.getInputStream();
+
+      conn = run(factory, new URL(buf.toString()));
+      InputStream in = conn.getInputStream();
       Credentials ts = new Credentials();
       dis = new DataInputStream(in);
       ts.readFields(dis);
-      for(Token<?> token: ts.getAllTokens()) {
+      for (Token<?> token : ts.getAllTokens()) {
         token.setKind(HftpFileSystem.TOKEN_KIND);
         SecurityUtil.setTokenService(token, serviceAddr);
       }
@@ -241,53 +244,70 @@ public class DelegationTokenFetcher {
     } catch (Exception e) {
       throw new IOException("Unable to obtain remote token", e);
     } finally {
-      if(dis != null) dis.close();
+      IOUtils.cleanup(LOG, dis);
+      if (conn != null) {
+        conn.disconnect();
+      }
     }
   }
 
+  /**
+   * Cancel a Delegation Token.
+   * @param nnAddr the NameNode's address
+   * @param tok the token to cancel
+   * @throws IOException
+   * @throws AuthenticationException
+   */
+  static public void cancelDelegationToken(URLConnectionFactory factory,
+      URI nnAddr, Token<DelegationTokenIdentifier> tok) throws IOException,
+      AuthenticationException {
+    StringBuilder buf = new StringBuilder(nnAddr.toString())
+        .append(CancelDelegationTokenServlet.PATH_SPEC).append("?")
+        .append(CancelDelegationTokenServlet.TOKEN).append("=")
+        .append(tok.encodeToUrlString());
+    HttpURLConnection conn = run(factory, new URL(buf.toString()));
+    conn.disconnect();
+  }
+
   /**
    * Renew a Delegation Token.
    * @param nnAddr the NameNode's address
    * @param tok the token to renew
    * @return the Date that the token will expire next.
    * @throws IOException
+   * @throws AuthenticationException
    */
-  static public long renewDelegationToken(String nnAddr,
-      Token<DelegationTokenIdentifier> tok
-  ) throws IOException {
-    StringBuilder buf = new StringBuilder();
-    buf.append(nnAddr);
-    buf.append(RenewDelegationTokenServlet.PATH_SPEC);
-    buf.append("?");
-    buf.append(RenewDelegationTokenServlet.TOKEN);
-    buf.append("=");
-    buf.append(tok.encodeToUrlString());
-    BufferedReader in = null;
+  static public long renewDelegationToken(URLConnectionFactory factory,
+      URI nnAddr, Token<DelegationTokenIdentifier> tok) throws IOException,
+      AuthenticationException {
+    StringBuilder buf = new StringBuilder(nnAddr.toString())
+        .append(RenewDelegationTokenServlet.PATH_SPEC).append("?")
+        .append(RenewDelegationTokenServlet.TOKEN).append("=")
+        .append(tok.encodeToUrlString());
+
     HttpURLConnection connection = null;
-    
+    BufferedReader in = null;
     try {
-      URL url = new URL(buf.toString());
-      connection = (HttpURLConnection) SecurityUtil.openSecureHttpConnection(url);
-      if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) {
-        throw new IOException("Error renewing token: " + 
-            connection.getResponseMessage());
-      }
-      in = new BufferedReader(
-          new InputStreamReader(connection.getInputStream(), Charsets.UTF_8));
+      connection = run(factory, new URL(buf.toString()));
+      in = new BufferedReader(new InputStreamReader(
+          connection.getInputStream(), Charsets.UTF_8));
       long result = Long.parseLong(in.readLine());
-      in.close();
       return result;
     } catch (IOException ie) {
       LOG.info("error in renew over HTTP", ie);
       IOException e = getExceptionFromResponse(connection);
 
-      IOUtils.cleanup(LOG, in);
-      if(e!=null) {
-        LOG.info("rethrowing exception from HTTP request: " + 
-                 e.getLocalizedMessage());
+      if (e != null) {
+        LOG.info("rethrowing exception from HTTP request: "
+            + e.getLocalizedMessage());
         throw e;
       }
       throw ie;
+    } finally {
+      IOUtils.cleanup(LOG, in);
+      if (connection != null) {
+        connection.disconnect();
+      }
     }
   }
 
@@ -339,43 +359,28 @@ public class DelegationTokenFetcher {
     return e;
   }
 
-  
-  /**
-   * Cancel a Delegation Token.
-   * @param nnAddr the NameNode's address
-   * @param tok the token to cancel
-   * @throws IOException
-   */
-  static public void cancelDelegationToken(String nnAddr,
-      Token<DelegationTokenIdentifier> tok
-  ) throws IOException {
-    StringBuilder buf = new StringBuilder();
-    buf.append(nnAddr);
-    buf.append(CancelDelegationTokenServlet.PATH_SPEC);
-    buf.append("?");
-    buf.append(CancelDelegationTokenServlet.TOKEN);
-    buf.append("=");
-    buf.append(tok.encodeToUrlString());
-    BufferedReader in = null;
-    HttpURLConnection connection=null;
+  private static HttpURLConnection run(URLConnectionFactory factory, URL url)
+      throws IOException, AuthenticationException {
+    HttpURLConnection conn = null;
+
     try {
-      URL url = new URL(buf.toString());
-      connection = (HttpURLConnection) SecurityUtil.openSecureHttpConnection(url);
-      if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) {
-        throw new IOException("Error cancelling token: " + 
-            connection.getResponseMessage());
+      conn = (HttpURLConnection) factory.openConnection(url, true);
+      if (conn.getResponseCode() != HttpURLConnection.HTTP_OK) {
+        String msg = conn.getResponseMessage();
+
+        throw new IOException("Error when dealing remote token: " + msg);
       }
     } catch (IOException ie) {
-      LOG.info("error in cancel over HTTP", ie);
-      IOException e = getExceptionFromResponse(connection);
+      LOG.info("Error when dealing remote token:", ie);
+      IOException e = getExceptionFromResponse(conn);
 
-      IOUtils.cleanup(LOG, in);
-      if(e!=null) {
-        LOG.info("rethrowing exception from HTTP request: " + 
-                 e.getLocalizedMessage());
+      if (e != null) {
+        LOG.info("rethrowing exception from HTTP request: "
+            + e.getLocalizedMessage());
         throw e;
       }
       throw ie;
     }
+    return conn;
   }
 }

+ 47 - 116
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/HftpFileSystem.java

@@ -31,7 +31,6 @@ import java.security.PrivilegedExceptionAction;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.TimeZone;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -50,20 +49,17 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
-import org.apache.hadoop.hdfs.web.ByteRangeInputStream.URLOpener;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.security.token.TokenRenewer;
-import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ServletUtil;
 import org.xml.sax.Attributes;
@@ -83,7 +79,9 @@ import org.xml.sax.helpers.XMLReaderFactory;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class HftpFileSystem extends FileSystem
-    implements DelegationTokenRenewer.Renewable {
+    implements DelegationTokenRenewer.Renewable, TokenAspect.TokenManagementDelegator {
+  public static final String SCHEME = "hftp";
+
   static {
     HttpURLConnection.setFollowRedirects(true);
   }
@@ -100,19 +98,13 @@ public class HftpFileSystem extends FileSystem
   public static final String HFTP_TIMEZONE = "UTC";
   public static final String HFTP_DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
 
+  private TokenAspect<HftpFileSystem> tokenAspect = new TokenAspect<HftpFileSystem>(this, TOKEN_KIND);
   private Token<?> delegationToken;
   private Token<?> renewToken;
-  private static final HftpDelegationTokenSelector hftpTokenSelector =
-      new HftpDelegationTokenSelector();
-
-  private DelegationTokenRenewer dtRenewer = null;
 
-  private synchronized void addRenewAction(final HftpFileSystem hftpFs) {
-    if (dtRenewer == null) {
-      dtRenewer = DelegationTokenRenewer.getInstance();
-    }
-
-    dtRenewer.addRenewAction(hftpFs);
+  @Override
+  public URI getCanonicalUri() {
+    return super.getCanonicalUri();
   }
 
   public static final SimpleDateFormat getDateFormat() {
@@ -177,7 +169,7 @@ public class HftpFileSystem extends FileSystem
    */
   @Override
   public String getScheme() {
-    return "hftp";
+    return SCHEME;
   }
 
   @Override
@@ -195,39 +187,10 @@ public class HftpFileSystem extends FileSystem
     }
 
     if (UserGroupInformation.isSecurityEnabled()) {
-      initDelegationToken();
-    }
-  }
-
-  protected void initDelegationToken() throws IOException {
-    // look for hftp token, then try hdfs
-    Token<?> token = selectDelegationToken(ugi);
-
-    // if we don't already have a token, go get one over https
-    boolean createdToken = false;
-    if (token == null) {
-      token = getDelegationToken(null);
-      createdToken = (token != null);
-    }
-
-    // we already had a token or getDelegationToken() didn't fail.
-    if (token != null) {
-      setDelegationToken(token);
-      if (createdToken) {
-        addRenewAction(this);
-        LOG.debug("Created new DT for " + token.getService());
-      } else {
-        LOG.debug("Found existing DT for " + token.getService());
-      }
+      tokenAspect.initDelegationToken(ugi);
     }
   }
 
-  protected Token<DelegationTokenIdentifier> selectDelegationToken(
-      UserGroupInformation ugi) {
-    return hftpTokenSelector.selectToken(nnUri, ugi.getTokens(), getConf());
-  }
-
-
   @Override
   public Token<?> getRenewToken() {
     return renewToken;
@@ -242,16 +205,19 @@ public class HftpFileSystem extends FileSystem
 
   @Override
   public synchronized <T extends TokenIdentifier> void setDelegationToken(Token<T> token) {
+    /**
+     * XXX The kind of the token has been changed by DelegationTokenFetcher. We
+     * use the token for renewal, since the reflection utilities needs the value
+     * of the kind field to correctly renew the token.
+     *
+     * For other operations, however, the client has to send a
+     * HDFS_DELEGATION_KIND token over the wire so that it can talk to Hadoop
+     * 0.20.3 clusters. Later releases fix this problem. See HDFS-5440 for more
+     * details.
+     */
     renewToken = token;
-    // emulate the 203 usage of the tokens
-    // by setting the kind and service as if they were hdfs tokens
     delegationToken = new Token<T>(token);
-    // NOTE: the remote nn must be configured to use hdfs
     delegationToken.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
-    // no need to change service because we aren't exactly sure what it
-    // should be.  we can guess, but it might be wrong if the local conf
-    // value is incorrect.  the service is a client side field, so the remote
-    // end does not care about the value
   }
 
   @Override
@@ -266,7 +232,7 @@ public class HftpFileSystem extends FileSystem
           final String nnHttpUrl = nnUri.toString();
           Credentials c;
           try {
-            c = DelegationTokenFetcher.getDTfromRemote(nnHttpUrl, renewer);
+            c = DelegationTokenFetcher.getDTfromRemote(connectionFactory, nnUri, renewer);
           } catch (IOException e) {
             if (e.getCause() instanceof ConnectException) {
               LOG.warn("Couldn't connect to " + nnHttpUrl +
@@ -350,6 +316,7 @@ public class HftpFileSystem extends FileSystem
     String tokenString = null;
     if (UserGroupInformation.isSecurityEnabled()) {
       synchronized (this) {
+        tokenAspect.ensureTokenInitialized();
         if (delegationToken != null) {
           tokenString = delegationToken.encodeToUrlString();
           return (query + JspHelper.getDelegationTokenUrlParam(tokenString));
@@ -419,9 +386,7 @@ public class HftpFileSystem extends FileSystem
   @Override
   public void close() throws IOException {
     super.close();
-    if (dtRenewer != null) {
-      dtRenewer.removeRenewAction(this); // blocks
-    }
+    tokenAspect.removeRenewAction();
   }
 
   /** Class to parse and store a listing reply from the server. */
@@ -696,67 +661,33 @@ public class HftpFileSystem extends FileSystem
     return cs != null? cs: super.getContentSummary(f);
   }
 
-  @InterfaceAudience.Private
-  public static class TokenManager extends TokenRenewer {
-
-    @Override
-    public boolean handleKind(Text kind) {
-      return kind.equals(TOKEN_KIND);
-    }
-
-    @Override
-    public boolean isManaged(Token<?> token) throws IOException {
-      return true;
-    }
-
-    protected String getUnderlyingProtocol() {
-      return "http";
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public long renew(Token<?> token,
-                      Configuration conf) throws IOException {
-      // update the kerberos credentials, if they are coming from a keytab
-      UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
-      InetSocketAddress serviceAddr = SecurityUtil.getTokenServiceAddr(token);
-      return
-        DelegationTokenFetcher.renewDelegationToken
-        (DFSUtil.createUri(getUnderlyingProtocol(), serviceAddr).toString(),
-         (Token<DelegationTokenIdentifier>) token);
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public void cancel(Token<?> token,
-                       Configuration conf) throws IOException {
-      // update the kerberos credentials, if they are coming from a keytab
-      UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
-      InetSocketAddress serviceAddr = SecurityUtil.getTokenServiceAddr(token);
-      DelegationTokenFetcher.cancelDelegationToken
-        (DFSUtil.createUri(getUnderlyingProtocol(), serviceAddr).toString(),
-         (Token<DelegationTokenIdentifier>) token);
+  @SuppressWarnings("unchecked")
+  @Override
+  public long renewDelegationToken(Token<?> token) throws IOException {
+    // update the kerberos credentials, if they are coming from a keytab
+    UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
+    InetSocketAddress serviceAddr = SecurityUtil.getTokenServiceAddr(token);
+    try {
+      return DelegationTokenFetcher.renewDelegationToken(connectionFactory,
+          DFSUtil.createUri(getUnderlyingProtocol(), serviceAddr),
+          (Token<DelegationTokenIdentifier>) token);
+    } catch (AuthenticationException e) {
+      throw new IOException(e);
     }
   }
 
-  private static class HftpDelegationTokenSelector
-  extends AbstractDelegationTokenSelector<DelegationTokenIdentifier> {
-    private static final DelegationTokenSelector hdfsTokenSelector =
-        new DelegationTokenSelector();
-
-    public HftpDelegationTokenSelector() {
-      super(TOKEN_KIND);
-    }
-
-    Token<DelegationTokenIdentifier> selectToken(URI nnUri,
-        Collection<Token<?>> tokens, Configuration conf) {
-      Token<DelegationTokenIdentifier> token =
-          selectToken(SecurityUtil.buildTokenService(nnUri), tokens);
-      if (token == null) {
-        // try to get a HDFS token
-        token = hdfsTokenSelector.selectToken(nnUri, tokens, conf);
-      }
-      return token;
+  @SuppressWarnings("unchecked")
+  @Override
+  public void cancelDelegationToken(Token<?> token) throws IOException {
+    // update the kerberos credentials, if they are coming from a keytab
+    UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
+    InetSocketAddress serviceAddr = SecurityUtil.getTokenServiceAddr(token);
+    try {
+      DelegationTokenFetcher.cancelDelegationToken(connectionFactory, DFSUtil
+          .createUri(getUnderlyingProtocol(), serviceAddr),
+          (Token<DelegationTokenIdentifier>) token);
+    } catch (AuthenticationException e) {
+      throw new IOException(e);
     }
   }
 }

+ 177 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java

@@ -0,0 +1,177 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.Collection;
+
+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.fs.DelegationTokenRenewer;
+import org.apache.hadoop.fs.DelegationTokenRenewer.Renewable;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
+import org.apache.hadoop.io.Text;
+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.TokenRenewer;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This class implements the aspects that relate to delegation tokens for all
+ * HTTP-based file system.
+ */
+final class TokenAspect<T extends FileSystem & Renewable> {
+  @InterfaceAudience.Private
+  public static class TokenManager extends TokenRenewer {
+
+    @Override
+    public void cancel(Token<?> token, Configuration conf) throws IOException {
+      getInstance(token, conf).cancelDelegationToken(token);
+    }
+
+    @Override
+    public boolean handleKind(Text kind) {
+      return kind.equals(HftpFileSystem.TOKEN_KIND)
+          || kind.equals(WebHdfsFileSystem.TOKEN_KIND);
+    }
+
+    @Override
+    public boolean isManaged(Token<?> token) throws IOException {
+      return true;
+    }
+
+    @Override
+    public long renew(Token<?> token, Configuration conf) throws IOException {
+      return getInstance(token, conf).renewDelegationToken(token);
+    }
+
+    private TokenManagementDelegator getInstance(Token<?> token,
+        Configuration conf) throws IOException {
+      final InetSocketAddress address = SecurityUtil.getTokenServiceAddr(token);
+      Text kind = token.getKind();
+      final URI uri;
+      if (kind.equals(HftpFileSystem.TOKEN_KIND)) {
+        uri = DFSUtil.createUri(HftpFileSystem.SCHEME, address);
+      } else if (kind.equals(WebHdfsFileSystem.TOKEN_KIND)) {
+        uri = DFSUtil.createUri(WebHdfsFileSystem.SCHEME, address);
+      } else {
+        throw new IllegalArgumentException("Unsupported scheme");
+      }
+      return (TokenManagementDelegator) FileSystem.get(uri, conf);
+    }
+  }
+
+  static class DTSelecorByKind extends
+      AbstractDelegationTokenSelector<DelegationTokenIdentifier> {
+    private static final DelegationTokenSelector selector = new DelegationTokenSelector();
+
+    public DTSelecorByKind(final Text kind) {
+      super(kind);
+    }
+
+    Token<DelegationTokenIdentifier> selectToken(URI nnUri,
+        Collection<Token<?>> tokens, Configuration conf) {
+      Token<DelegationTokenIdentifier> token = selectToken(
+          SecurityUtil.buildTokenService(nnUri), tokens);
+      if (token == null) {
+        token = selector.selectToken(nnUri, tokens, conf);
+      }
+      return token;
+    }
+  }
+
+  /**
+   * Callbacks for token management
+   */
+  interface TokenManagementDelegator {
+    void cancelDelegationToken(final Token<?> token) throws IOException;
+
+    URI getCanonicalUri();
+
+    long renewDelegationToken(final Token<?> token) throws IOException;
+  }
+
+  private DelegationTokenRenewer.RenewAction<?> action;
+  private DelegationTokenRenewer dtRenewer = null;
+  private final DTSelecorByKind dtSelector;
+  private final T fs;
+  private boolean hasInitedToken;
+  private final Log LOG;
+
+  TokenAspect(T fs, final Text kind) {
+    this.LOG = LogFactory.getLog(fs.getClass());
+    this.fs = fs;
+    this.dtSelector = new DTSelecorByKind(kind);
+  }
+
+  synchronized void ensureTokenInitialized() throws IOException {
+    // we haven't inited yet, or we used to have a token but it expired
+    if (!hasInitedToken || (action != null && !action.isValid())) {
+      //since we don't already have a token, go get one
+      Token<?> token = fs.getDelegationToken(null);
+      // security might be disabled
+      if (token != null) {
+        fs.setDelegationToken(token);
+        addRenewAction(fs);
+        LOG.debug("Created new DT for " + token.getService());
+      }
+      hasInitedToken = true;
+    }
+  }
+
+  synchronized void initDelegationToken(UserGroupInformation ugi) {
+    Token<?> token = selectDelegationToken(ugi);
+    if (token != null) {
+      LOG.debug("Found existing DT for " + token.getService());
+      fs.setDelegationToken(token);
+      hasInitedToken = true;
+    }
+  }
+
+  synchronized void removeRenewAction() throws IOException {
+    if (dtRenewer != null) {
+      dtRenewer.removeRenewAction(fs);
+    }
+  }
+
+  @VisibleForTesting
+  Token<DelegationTokenIdentifier> selectDelegationToken(
+      UserGroupInformation ugi) {
+    return dtSelector.selectToken(
+        ((TokenManagementDelegator)fs).getCanonicalUri(), ugi.getTokens(),
+        fs.getConf());
+  }
+
+  private synchronized void addRenewAction(final T webhdfs) {
+    if (dtRenewer == null) {
+      dtRenewer = DelegationTokenRenewer.getInstance();
+    }
+
+    action = dtRenewer.addRenewAction(webhdfs);
+  }
+}

+ 17 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java

@@ -27,7 +27,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
@@ -77,24 +77,28 @@ public class URLConnectionFactory {
    * @throws IOException
    */
   public URLConnection openConnection(URL url) throws IOException {
-    URLConnection connection = url.openConnection();
-    if (connection instanceof HttpURLConnection) {
-      connConfigurator.configure((HttpURLConnection) connection);
+    try {
+      return openConnection(url, false);
+    } catch (AuthenticationException e) {
+      // Unreachable
+      return null;
     }
-    return connection;
   }
 
   /**
    * Opens a url with read and connect timeouts
    *
-   * @param url URL to open
+   * @param url
+   *          URL to open
+   * @param isSpnego
+   *          whether the url should be authenticated via SPNEGO
    * @return URLConnection
    * @throws IOException
    * @throws AuthenticationException
    */
-  public URLConnection openConnection(HttpOpParam.Op op, URL url)
+  public URLConnection openConnection(URL url, boolean isSpnego)
       throws IOException, AuthenticationException {
-    if (op.getRequireAuth()) {
+    if (isSpnego) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("open AuthenticatedURL connection" + url);
       }
@@ -106,7 +110,11 @@ public class URLConnectionFactory {
       if (LOG.isDebugEnabled()) {
         LOG.debug("open URL connection");
       }
-      return openConnection(url);
+      URLConnection connection = url.openConnection();
+      if (connection instanceof HttpURLConnection) {
+        connConfigurator.configure((HttpURLConnection) connection);
+      }
+      return connection;
     }
   }
 

+ 15 - 64
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -30,7 +30,6 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
-import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.StringTokenizer;
@@ -56,8 +55,8 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+import org.apache.hadoop.hdfs.web.TokenAspect.DTSelecorByKind;
 import org.apache.hadoop.hdfs.web.resources.AccessTimeParam;
 import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
 import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
@@ -96,8 +95,6 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.security.token.TokenRenewer;
-import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
 import org.apache.hadoop.util.Progressable;
 import org.mortbay.util.ajax.JSON;
 
@@ -107,7 +104,7 @@ import com.google.common.collect.Lists;
 
 /** A FileSystem for HDFS over the web. */
 public class WebHdfsFileSystem extends FileSystem
-    implements DelegationTokenRenewer.Renewable {
+    implements DelegationTokenRenewer.Renewable, TokenAspect.TokenManagementDelegator {
   public static final Log LOG = LogFactory.getLog(WebHdfsFileSystem.class);
   /** File System URI: {SCHEME}://namenode:port/path/to/file */
   public static final String SCHEME = "webhdfs";
@@ -122,13 +119,18 @@ public class WebHdfsFileSystem extends FileSystem
   /** Delegation token kind */
   public static final Text TOKEN_KIND = new Text("WEBHDFS delegation");
   /** Token selector */
-  public static final WebHdfsDelegationTokenSelector DT_SELECTOR
-      = new WebHdfsDelegationTokenSelector();
+  public static final DTSelecorByKind DT_SELECTOR
+      = new DTSelecorByKind(TOKEN_KIND);
 
   private DelegationTokenRenewer dtRenewer = null;
   @VisibleForTesting
   DelegationTokenRenewer.RenewAction<?> action;
 
+  @Override
+  public URI getCanonicalUri() {
+    return super.getCanonicalUri();
+  }
+
   @VisibleForTesting
   protected synchronized void addRenewAction(final WebHdfsFileSystem webhdfs) {
     if (dtRenewer == null) {
@@ -142,7 +144,6 @@ public class WebHdfsFileSystem extends FileSystem
   public static boolean isEnabled(final Configuration conf, final Log log) {
     final boolean b = conf.getBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY,
         DFSConfigKeys.DFS_WEBHDFS_ENABLED_DEFAULT);
-    log.info(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY + " = " + b);
     return b;
   }
 
@@ -490,7 +491,8 @@ public class WebHdfsFileSystem extends FileSystem
         throws IOException {
       final HttpURLConnection conn;
       try {
-        conn = (HttpURLConnection) connectionFactory.openConnection(op, url);
+        conn = (HttpURLConnection) connectionFactory.openConnection(url,
+            op.getRequireAuth());
       } catch (AuthenticationException e) {
         throw new IOException(e);
       }
@@ -986,7 +988,8 @@ public class WebHdfsFileSystem extends FileSystem
     }
   }
 
-  private synchronized long renewDelegationToken(final Token<?> token
+  @Override
+  public synchronized long renewDelegationToken(final Token<?> token
       ) throws IOException {
     final HttpOpParam.Op op = PutOpParam.Op.RENEWDELEGATIONTOKEN;
     TokenArgumentParam dtargParam = new TokenArgumentParam(
@@ -995,7 +998,8 @@ public class WebHdfsFileSystem extends FileSystem
     return (Long) m.get("long");
   }
 
-  private synchronized void cancelDelegationToken(final Token<?> token
+  @Override
+  public synchronized void cancelDelegationToken(final Token<?> token
       ) throws IOException {
     final HttpOpParam.Op op = PutOpParam.Op.CANCELDELEGATIONTOKEN;
     TokenArgumentParam dtargParam = new TokenArgumentParam(
@@ -1041,57 +1045,4 @@ public class WebHdfsFileSystem extends FileSystem
     final Map<?, ?> m = run(op, p);
     return JsonUtil.toMD5MD5CRC32FileChecksum(m);
   }
-
-  /** Delegation token renewer. */
-  public static class DtRenewer extends TokenRenewer {
-    @Override
-    public boolean handleKind(Text kind) {
-      return kind.equals(TOKEN_KIND);
-    }
-  
-    @Override
-    public boolean isManaged(Token<?> token) throws IOException {
-      return true;
-    }
-
-    private static WebHdfsFileSystem getWebHdfs(
-        final Token<?> token, final Configuration conf) throws IOException {
-      
-      final InetSocketAddress nnAddr = SecurityUtil.getTokenServiceAddr(token);
-      final URI uri = DFSUtil.createUri(WebHdfsFileSystem.SCHEME, nnAddr);
-      return (WebHdfsFileSystem)FileSystem.get(uri, conf);
-    }
-
-    @Override
-    public long renew(final Token<?> token, final Configuration conf
-        ) throws IOException, InterruptedException {
-      return getWebHdfs(token, conf).renewDelegationToken(token);
-    }
-  
-    @Override
-    public void cancel(final Token<?> token, final Configuration conf
-        ) throws IOException, InterruptedException {
-      getWebHdfs(token, conf).cancelDelegationToken(token);
-    }
-  }
-  
-  private static class WebHdfsDelegationTokenSelector
-  extends AbstractDelegationTokenSelector<DelegationTokenIdentifier> {
-    private static final DelegationTokenSelector hdfsTokenSelector =
-        new DelegationTokenSelector();
-    
-    public WebHdfsDelegationTokenSelector() {
-      super(TOKEN_KIND);
-    }
-    
-    Token<DelegationTokenIdentifier> selectToken(URI nnUri,
-        Collection<Token<?>> tokens, Configuration conf) {
-      Token<DelegationTokenIdentifier> token =
-          selectToken(SecurityUtil.buildTokenService(nnUri), tokens);
-      if (token == null) {
-        token = hdfsTokenSelector.selectToken(nnUri, tokens, conf); 
-      }
-      return token;
-    }
-  }
 }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer

@@ -13,5 +13,4 @@
 #
 org.apache.hadoop.hdfs.DFSClient$Renewer
 org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier$Renewer
-org.apache.hadoop.hdfs.web.HftpFileSystem$TokenManager
-org.apache.hadoop.hdfs.web.WebHdfsFileSystem$DtRenewer
+org.apache.hadoop.hdfs.web.TokenAspect$TokenManager

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

@@ -1516,6 +1516,18 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.namenode.path.based.cache.retry.interval.ms</name>
+  <value>60000</value>
+  <description>
+    When the NameNode needs to uncache something that is cached, or cache
+    something that is not cached, it must direct the DataNodes to do so by
+    sending a DNA_CACHE or DNA_UNCACHE command in response to a DataNode
+    heartbeat.  This parameter controls how frequently the NameNode will
+    resend these commands.
+  </description>
+</property>
+
 <property>
   <name>dfs.datanode.fsdatasetcache.max.threads.per.volume</name>
   <value>4</value>

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html

@@ -30,6 +30,12 @@
 </div>
 <div id="panel"></div>
 </div>
+<div class="row">
+<hr />
+<div class="col-xs-2"><p><a href="http://hadoop.apache.org/core">Hadoop</a>, 2013.</p></div>
+<div class="col-xs-1 pull-right"><a style="color: #ddd" href="dfshealth.jsp">Legacy UI</a></div>
+</div>
+
 <script type="text/javascript" src="//ajax.googleapis.com/ajax/libs/jquery/2.0.3/jquery.min.js">
 </script><script type="text/javascript" src="//netdna.bootstrapcdn.com/bootstrap/3.0.0/js/bootstrap.min.js">
 </script><script type="text/javascript" src="/static/dust-full-2.0.0.min.js">
@@ -37,7 +43,5 @@
 </script><script type="text/javascript" src="dfs-dust.js">
 </script><script type="text/javascript" src="dfshealth.js">
 </script>
-<hr />
-<p><a href="http://hadoop.apache.org/core">Hadoop</a>, 2013.</p>
 </body>
 </html>

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp

@@ -70,6 +70,6 @@
 <hr/>
 <h3>Startup Progress</h3>
 <% healthjsp.generateStartupProgress(out, nn.getStartupProgress()); %>
-<%
-out.println(ServletUtil.htmlFooter());
-%>
+<hr/><p><a href="http://hadoop.apache.org/core">Hadoop</a>, 2013.&nbsp;<a href="dfshealth.html">New UI</a></p>
+</body>
+</html>

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/index.html

@@ -14,22 +14,22 @@
    See the License for the specific language governing permissions and
    limitations under the License.
 -->
-<meta HTTP-EQUIV="REFRESH" content="0;url=dfshealth.jsp"/>
-<html>
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
+    "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
+<html xmlns="http://www.w3.org/1999/xhtml">
 <head>
+<meta http-equiv="REFRESH" content="1;url=dfshealth.jsp" />
 <title>Hadoop Administration</title>
 </head>
-
 <body>
-
+<script type="text/javascript">
+//<![CDATA[
+window.location.href='dfshealth.html';
+//]]>
+</script>
 <h1>Hadoop Administration</h1>
-
 <ul>
-
 <li><a href="dfshealth.jsp">DFS Health/Status</a></li>
-
 </ul>
-
 </body>
-
 </html>

+ 55 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java

@@ -26,10 +26,14 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.mockito.Mockito;
+import org.mockito.stubbing.Answer;
+
 /**
  * This tests pipeline recovery related client protocol works correct or not.
  */
@@ -112,4 +116,55 @@ public class TestClientProtocolForPipelineRecovery {
       cluster.shutdown();
     }
   }
+
+  /** Test whether corrupt replicas are detected correctly during pipeline
+   * recoveries.
+   */
+  @Test
+  public void testPipelineRecoveryForLastBlock() throws IOException {
+    DFSClientFaultInjector faultInjector
+        = Mockito.mock(DFSClientFaultInjector.class);
+    DFSClientFaultInjector oldInjector = DFSClientFaultInjector.instance;
+    DFSClientFaultInjector.instance = faultInjector;
+    Configuration conf = new HdfsConfiguration();
+
+    conf.setInt(DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY, 3);
+    MiniDFSCluster cluster = null;
+
+    try {
+      int numDataNodes = 3;
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
+      cluster.waitActive();
+      FileSystem fileSys = cluster.getFileSystem();
+
+      Path file = new Path("dataprotocol1.dat");
+      Mockito.when(faultInjector.failPacket()).thenReturn(true);
+      try {
+        DFSTestUtil.createFile(fileSys, file, 1L, (short)numDataNodes, 0L);
+      } catch (IOException e) {
+        // completeFile() should fail.
+        Assert.assertTrue(e.getMessage().startsWith("Unable to close file"));
+        return;
+      }
+
+      // At this point, NN let data corruption to happen. 
+      // Before failing test, try reading the file. It should fail.
+      FSDataInputStream in = fileSys.open(file);
+      try {
+        int c = in.read();
+        // Test will fail with BlockMissingException if NN does not update the
+        // replica state based on the latest report.
+      } catch (org.apache.hadoop.hdfs.BlockMissingException bme) {
+        Assert.fail("Block is missing because the file was closed with"
+            + " corrupt replicas.");
+      }
+      Assert.fail("The file was closed with corrupt replicas, but read still"
+          + " works!");
+    } finally {
+      DFSClientFaultInjector.instance = oldInjector;
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

+ 16 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java

@@ -86,6 +86,9 @@ public class TestQuota {
     // Space quotas
     final int DEFAULT_BLOCK_SIZE = 512;
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
+    // Make it relinquish locks. When run serially, the result should
+    // be identical.
+    conf.setInt(DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_KEY, 2);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     final FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(),
@@ -350,6 +353,7 @@ public class TestQuota {
       }
       assertTrue(hasException);
 
+      assertEquals(4, cluster.getNamesystem().getFSDirectory().getYieldCount());
     } finally {
       cluster.shutdown();
     }
@@ -360,6 +364,9 @@ public class TestQuota {
   @Test
   public void testNamespaceCommands() throws Exception {
     final Configuration conf = new HdfsConfiguration();
+    // Make it relinquish locks. When run serially, the result should
+    // be identical.
+    conf.setInt(DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_KEY, 2);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     final FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(),
@@ -515,6 +522,7 @@ public class TestQuota {
       c = dfs.getContentSummary(quotaDir1);
       assertEquals(c.getDirectoryCount(), 6);
       assertEquals(c.getQuota(), 6);
+      assertEquals(14, cluster.getNamesystem().getFSDirectory().getYieldCount());
     } finally {
       cluster.shutdown();
     }
@@ -532,6 +540,9 @@ public class TestQuota {
     // set a smaller block size so that we can test with smaller 
     // diskspace quotas
     conf.set(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, "512");
+    // Make it relinquish locks. When run serially, the result should
+    // be identical.
+    conf.setInt(DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_KEY, 2);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     final FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(),
@@ -764,6 +775,7 @@ public class TestQuota {
       assertEquals(c.getSpaceConsumed(),
           (sizeFactorA + sizeFactorB + sizeFactorC) * fileSpace);
 
+      assertEquals(20, cluster.getNamesystem().getFSDirectory().getYieldCount());
     } finally {
       cluster.shutdown();
     }
@@ -905,6 +917,9 @@ public class TestQuota {
     final int BLOCK_SIZE = 6 * 1024;
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
+    // Make it relinquish locks. When run serially, the result should
+    // be identical.
+    conf.setInt(DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_KEY, 2);
     MiniDFSCluster cluster = 
       new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     cluster.waitActive();
@@ -971,6 +986,7 @@ public class TestQuota {
         exceededQuota = true;
       }
       assertTrue("Quota not exceeded", exceededQuota);
+      assertEquals(2, cluster.getNamesystem().getFSDirectory().getYieldCount());
     } finally {
       cluster.shutdown();
     }

+ 63 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java

@@ -28,8 +28,10 @@ import static org.mockito.Mockito.doReturn;
 
 import java.io.FileInputStream;
 import java.io.IOException;
-import java.nio.MappedByteBuffer;
+import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
+import java.util.HashSet;
+import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -49,7 +51,6 @@ import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetCache.PageRounder;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MappableBlock;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MappableBlock.Mlocker;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
@@ -60,6 +61,7 @@ import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.io.nativeio.NativeIO.POSIX.CacheManipulator;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.MetricsAsserts;
@@ -87,8 +89,7 @@ public class TestFsDatasetCache {
   private static FsDatasetSpi<?> fsd;
   private static DatanodeProtocolClientSideTranslatorPB spyNN;
   private static PageRounder rounder = new PageRounder();
-
-  private Mlocker mlocker;
+  private static CacheManipulator prevCacheManipulator;
 
   @Before
   public void setUp() throws Exception {
@@ -96,6 +97,8 @@ public class TestFsDatasetCache {
     assumeTrue(NativeIO.getMemlockLimit() >= CACHE_CAPACITY);
     conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_KEY, true);
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_RETRY_INTERVAL_MS,
+        500);
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     conf.setLong(DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
         CACHE_CAPACITY);
@@ -113,8 +116,19 @@ public class TestFsDatasetCache {
     fsd = dn.getFSDataset();
 
     spyNN = DataNodeTestUtils.spyOnBposToNN(dn, nn);
-    // Save the current mlocker and replace it at the end of the test
-    mlocker = MappableBlock.mlocker;
+
+    prevCacheManipulator = NativeIO.POSIX.cacheManipulator;
+
+    // Save the current CacheManipulator and replace it at the end of the test
+    // Stub out mlock calls to avoid failing when not enough memory is lockable
+    // by the operating system.
+    NativeIO.POSIX.cacheManipulator = new CacheManipulator() {
+      @Override
+      public void mlock(String identifier,
+          ByteBuffer mmap, long length) throws IOException {
+        LOG.info("mlocking " + identifier);
+      }
+    };
   }
 
   @After
@@ -125,8 +139,8 @@ public class TestFsDatasetCache {
     if (cluster != null) {
       cluster.shutdown();
     }
-    // Restore the original mlocker
-    MappableBlock.mlocker = mlocker;
+    // Restore the original CacheManipulator
+    NativeIO.POSIX.cacheManipulator = prevCacheManipulator;
   }
 
   private static void setHeartbeatResponse(DatanodeCommand[] cmds)
@@ -214,8 +228,7 @@ public class TestFsDatasetCache {
     return expected;
   }
 
-  @Test(timeout=600000)
-  public void testCacheAndUncacheBlock() throws Exception {
+  private void testCacheAndUncacheBlock() throws Exception {
     LOG.info("beginning testCacheAndUncacheBlock");
     final int NUM_BLOCKS = 5;
 
@@ -268,6 +281,42 @@ public class TestFsDatasetCache {
     LOG.info("finishing testCacheAndUncacheBlock");
   }
 
+  @Test(timeout=600000)
+  public void testCacheAndUncacheBlockSimple() throws Exception {
+    testCacheAndUncacheBlock();
+  }
+
+  /**
+   * Run testCacheAndUncacheBlock with some failures injected into the mlock
+   * call.  This tests the ability of the NameNode to resend commands.
+   */
+  @Test(timeout=600000)
+  public void testCacheAndUncacheBlockWithRetries() throws Exception {
+    CacheManipulator prevCacheManipulator = NativeIO.POSIX.cacheManipulator;
+    
+    try {
+      NativeIO.POSIX.cacheManipulator = new CacheManipulator() {
+        private final Set<String> seenIdentifiers = new HashSet<String>();
+        
+        @Override
+        public void mlock(String identifier,
+            ByteBuffer mmap, long length) throws IOException {
+          if (seenIdentifiers.contains(identifier)) {
+            // mlock succeeds the second time.
+            LOG.info("mlocking " + identifier);
+            return;
+          }
+          seenIdentifiers.add(identifier);
+          throw new IOException("injecting IOException during mlock of " +
+              identifier);
+        }
+      };
+      testCacheAndUncacheBlock();
+    } finally {
+      NativeIO.POSIX.cacheManipulator = prevCacheManipulator;
+    }
+  }
+
   @Test(timeout=600000)
   public void testFilesExceedMaxLockedMemory() throws Exception {
     LOG.info("beginning testFilesExceedMaxLockedMemory");
@@ -357,10 +406,11 @@ public class TestFsDatasetCache {
     assertEquals("Unexpected cache capacity", CACHE_CAPACITY, cacheCapacity);
     assertEquals("Unexpected amount of cache used", current, cacheUsed);
 
-    MappableBlock.mlocker = new MappableBlock.Mlocker() {
+    NativeIO.POSIX.cacheManipulator = new NativeIO.POSIX.CacheManipulator() {
       @Override
-      public void mlock(MappedByteBuffer mmap, long length) throws IOException {
-        LOG.info("An mlock operation is starting.");
+      public void mlock(String identifier,
+          ByteBuffer mmap, long length) throws IOException {
+        LOG.info("An mlock operation is starting on " + identifier);
         try {
           Thread.sleep(3000);
         } catch (InterruptedException e) {

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCorruptFilesJsp.java

@@ -93,6 +93,10 @@ public class TestCorruptFilesJsp  {
         in.close();
       }
 
+      try {
+        Thread.sleep(3000); // Wait for block reports. They shouldn't matter.
+      } catch (InterruptedException ie) {}
+
       // verify if all corrupt files were reported to NN
       badFiles = namenode.getNamesystem().listCorruptFileBlocks("/", null);
       assertTrue("Expecting 3 corrupt files, but got " + badFiles.size(),

+ 44 - 28
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathBasedCacheRequests.java

@@ -33,6 +33,7 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.nio.MappedByteBuffer;
 import java.security.PrivilegedExceptionAction;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.LinkedList;
@@ -60,6 +61,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBl
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MappableBlock;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.io.nativeio.NativeIO.POSIX.CacheManipulator;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -81,15 +83,7 @@ public class TestPathBasedCacheRequests {
   static private MiniDFSCluster cluster;
   static private DistributedFileSystem dfs;
   static private NamenodeProtocols proto;
-
-  static {
-    MappableBlock.mlocker = new MappableBlock.Mlocker() {
-      @Override
-      public void mlock(MappedByteBuffer mmap, long length) throws IOException {
-        // Stubbed out for testing
-      }
-    };
-  }
+  static private CacheManipulator prevCacheManipulator;
 
   @Before
   public void setup() throws Exception {
@@ -101,6 +95,18 @@ public class TestPathBasedCacheRequests {
     cluster.waitActive();
     dfs = cluster.getFileSystem();
     proto = cluster.getNameNodeRpc();
+    prevCacheManipulator = NativeIO.POSIX.cacheManipulator;
+
+    // Save the current CacheManipulator and replace it at the end of the test
+    // Stub out mlock calls to avoid failing when not enough memory is lockable
+    // by the operating system.
+    NativeIO.POSIX.cacheManipulator = new CacheManipulator() {
+      @Override
+      public void mlock(String identifier,
+          ByteBuffer mmap, long length) throws IOException {
+        LOG.info("mlocking " + identifier);
+      }
+    };
   }
 
   @After
@@ -108,6 +114,8 @@ public class TestPathBasedCacheRequests {
     if (cluster != null) {
       cluster.shutdown();
     }
+    // Restore the original CacheManipulator
+    NativeIO.POSIX.cacheManipulator = prevCacheManipulator;
   }
 
   @Test(timeout=60000)
@@ -552,8 +560,8 @@ public class TestPathBasedCacheRequests {
    * @throws Exception
    */
   private static void waitForCachedBlocks(NameNode nn,
-      final int expectedCachedBlocks, final int expectedCachedReplicas) 
-          throws Exception {
+      final int expectedCachedBlocks, final int expectedCachedReplicas,
+      final String logString) throws Exception {
     final FSNamesystem namesystem = nn.getNamesystem();
     final CacheManager cacheManager = namesystem.getCacheManager();
     LOG.info("Waiting for " + expectedCachedBlocks + " blocks with " +
@@ -581,9 +589,9 @@ public class TestPathBasedCacheRequests {
             (numCachedReplicas == expectedCachedReplicas)) {
           return true;
         } else {
-          LOG.info("cached blocks: have " + numCachedBlocks +
-              " / " + expectedCachedBlocks);
-          LOG.info("cached replicas: have " + numCachedReplicas +
+          LOG.info(logString + " cached blocks: have " + numCachedBlocks +
+              " / " + expectedCachedBlocks + ".  " +
+              "cached replicas: have " + numCachedReplicas +
               " / " + expectedCachedReplicas);
           return false;
         }
@@ -681,7 +689,7 @@ public class TestPathBasedCacheRequests {
         paths.add(p.toUri().getPath());
       }
       // Check the initial statistics at the namenode
-      waitForCachedBlocks(namenode, 0, 0);
+      waitForCachedBlocks(namenode, 0, 0, "testWaitForCachedReplicas:0");
       // Cache and check each path in sequence
       int expected = 0;
       for (int i=0; i<numFiles; i++) {
@@ -692,7 +700,8 @@ public class TestPathBasedCacheRequests {
               build();
         nnRpc.addPathBasedCacheDirective(directive);
         expected += numBlocksPerFile;
-        waitForCachedBlocks(namenode, expected, expected);
+        waitForCachedBlocks(namenode, expected, expected,
+            "testWaitForCachedReplicas:1");
       }
       // Uncache and check each path in sequence
       RemoteIterator<PathBasedCacheDirective> entries =
@@ -701,7 +710,8 @@ public class TestPathBasedCacheRequests {
         PathBasedCacheDirective directive = entries.next();
         nnRpc.removePathBasedCacheDirective(directive.getId());
         expected -= numBlocksPerFile;
-        waitForCachedBlocks(namenode, expected, expected);
+        waitForCachedBlocks(namenode, expected, expected,
+            "testWaitForCachedReplicas:2");
       }
     } finally {
       cluster.shutdown();
@@ -735,7 +745,8 @@ public class TestPathBasedCacheRequests {
         paths.add(p.toUri().getPath());
       }
       // Check the initial statistics at the namenode
-      waitForCachedBlocks(namenode, 0, 0);
+      waitForCachedBlocks(namenode, 0, 0,
+          "testAddingPathBasedCacheDirectivesWhenCachingIsDisabled:0");
       // Cache and check each path in sequence
       int expected = 0;
       for (int i=0; i<numFiles; i++) {
@@ -745,10 +756,12 @@ public class TestPathBasedCacheRequests {
               setPool(pool).
               build();
         dfs.addPathBasedCacheDirective(directive);
-        waitForCachedBlocks(namenode, expected, 0);
+        waitForCachedBlocks(namenode, expected, 0,
+          "testAddingPathBasedCacheDirectivesWhenCachingIsDisabled:1");
       }
       Thread.sleep(20000);
-      waitForCachedBlocks(namenode, expected, 0);
+      waitForCachedBlocks(namenode, expected, 0,
+          "testAddingPathBasedCacheDirectivesWhenCachingIsDisabled:2");
     } finally {
       cluster.shutdown();
     }
@@ -781,7 +794,8 @@ public class TestPathBasedCacheRequests {
         FileSystemTestHelper.createFile(dfs, path, numBlocksPerFile,
             (int)BLOCK_SIZE, (short)3, false);
       }
-      waitForCachedBlocks(namenode, 0, 0);
+      waitForCachedBlocks(namenode, 0, 0,
+          "testWaitForCachedReplicasInDirectory:0");
       // cache entire directory
       long id = dfs.addPathBasedCacheDirective(
             new PathBasedCacheDirective.Builder().
@@ -789,10 +803,12 @@ public class TestPathBasedCacheRequests {
               setReplication((short)2).
               setPool(pool).
               build());
-      waitForCachedBlocks(namenode, 4, 8);
+      waitForCachedBlocks(namenode, 4, 8,
+          "testWaitForCachedReplicasInDirectory:1");
       // remove and watch numCached go to 0
       dfs.removePathBasedCacheDirective(id);
-      waitForCachedBlocks(namenode, 0, 0);
+      waitForCachedBlocks(namenode, 0, 0,
+          "testWaitForCachedReplicasInDirectory:2");
     } finally {
       cluster.shutdown();
     }
@@ -830,7 +846,7 @@ public class TestPathBasedCacheRequests {
         FileSystemTestHelper.createFile(dfs, path, numBlocksPerFile,
             (int)BLOCK_SIZE, (short)3, false);
       }
-      waitForCachedBlocks(namenode, 0, 0);
+      waitForCachedBlocks(namenode, 0, 0, "testReplicationFactor:0");
       checkNumCachedReplicas(dfs, paths, 0, 0);
       // cache directory
       long id = dfs.addPathBasedCacheDirective(
@@ -839,7 +855,7 @@ public class TestPathBasedCacheRequests {
             setReplication((short)1).
             setPool(pool).
             build());
-      waitForCachedBlocks(namenode, 4, 4);
+      waitForCachedBlocks(namenode, 4, 4, "testReplicationFactor:1");
       checkNumCachedReplicas(dfs, paths, 4, 4);
       // step up the replication factor
       for (int i=2; i<=3; i++) {
@@ -848,7 +864,7 @@ public class TestPathBasedCacheRequests {
             setId(id).
             setReplication((short)i).
             build());
-        waitForCachedBlocks(namenode, 4, 4*i);
+        waitForCachedBlocks(namenode, 4, 4*i, "testReplicationFactor:2");
         checkNumCachedReplicas(dfs, paths, 4, 4*i);
       }
       // step it down
@@ -858,12 +874,12 @@ public class TestPathBasedCacheRequests {
             setId(id).
             setReplication((short)i).
             build());
-        waitForCachedBlocks(namenode, 4, 4*i);
+        waitForCachedBlocks(namenode, 4, 4*i, "testReplicationFactor:3");
         checkNumCachedReplicas(dfs, paths, 4, 4*i);
       }
       // remove and watch numCached go to 0
       dfs.removePathBasedCacheDirective(id);
-      waitForCachedBlocks(namenode, 0, 0);
+      waitForCachedBlocks(namenode, 0, 0, "testReplicationFactor:4");
       checkNumCachedReplicas(dfs, paths, 0, 0);
     } finally {
       cluster.shutdown();

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

@@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -47,7 +48,10 @@ import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiffList;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.io.IOUtils;
@@ -949,4 +953,54 @@ public class TestSnapshotDeletion {
     psOut.close();
     out.close();
   }
+
+  /*
+   * OP_DELETE_SNAPSHOT edits op was not decrementing the safemode threshold on
+   * restart in HA mode. HDFS-5504
+   */
+  @Test(timeout = 60000)
+  public void testHANNRestartAfterSnapshotDeletion() throws Exception {
+    hdfs.close();
+    cluster.shutdown();
+    conf = new Configuration();
+    cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(1)
+        .build();
+    cluster.transitionToActive(0);
+    // stop the standby namenode
+    NameNode snn = cluster.getNameNode(1);
+    snn.stop();
+
+    hdfs = (DistributedFileSystem) HATestUtil
+        .configureFailoverFs(cluster, conf);
+    Path dir = new Path("/dir");
+    Path subDir = new Path(dir, "sub");
+    hdfs.mkdirs(dir);
+    hdfs.allowSnapshot(dir);
+    for (int i = 0; i < 5; i++) {
+      DFSTestUtil.createFile(hdfs, new Path(subDir, "" + i), 100, (short) 1,
+          1024L);
+    }
+
+    // take snapshot
+    hdfs.createSnapshot(dir, "s0");
+
+    // delete the subdir
+    hdfs.delete(subDir, true);
+
+    // roll the edit log
+    NameNode ann = cluster.getNameNode(0);
+    ann.getRpcServer().rollEditLog();
+
+    hdfs.deleteSnapshot(dir, "s0");
+    // wait for the blocks deletion at namenode
+    Thread.sleep(2000);
+
+    NameNodeAdapter.abortEditLogs(ann);
+    cluster.restartNameNode(0, false);
+    cluster.transitionToActive(0);
+
+    // wait till the cluster becomes active
+    cluster.waitClusterUp();
+  }
 }

+ 19 - 45
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHftpDelegationToken.java

@@ -22,7 +22,6 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY
 import static org.junit.Assert.*;
 
 import java.io.IOException;
-import java.lang.reflect.Field;
 import java.net.ServerSocket;
 import java.net.Socket;
 import java.net.URI;
@@ -40,6 +39,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
 
 public class TestHftpDelegationToken {
 
@@ -71,9 +71,8 @@ public class TestHftpDelegationToken {
 	});
     assertSame("wrong kind of file system", HftpFileSystem.class,
                  fs.getClass());
-    Field renewToken = HftpFileSystem.class.getDeclaredField("renewToken");
-    renewToken.setAccessible(true);
-    assertSame("wrong token", token, renewToken.get(fs));
+    assertSame("wrong token", token,
+        Whitebox.getInternalState(fs, "renewToken"));
   }
 
   @Test
@@ -81,7 +80,7 @@ public class TestHftpDelegationToken {
     SecurityUtilTestHelper.setTokenServiceUseIp(true);
 
     Configuration conf = new Configuration();
-    conf.setClass("fs.hftp.impl", MyHftpFileSystem.class, FileSystem.class);
+    conf.setClass("fs.hftp.impl", HftpFileSystem.class, FileSystem.class);
 
     int httpPort = 80;
     int httpsPort = 443;
@@ -90,21 +89,21 @@ public class TestHftpDelegationToken {
 
     // test with implicit default port
     URI fsUri = URI.create("hftp://localhost");
-    MyHftpFileSystem fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
+    HftpFileSystem fs = (HftpFileSystem) FileSystem.newInstance(fsUri, conf);
     assertEquals(httpPort, fs.getCanonicalUri().getPort());
     checkTokenSelection(fs, httpPort, conf);
 
     // test with explicit default port
     // Make sure it uses the port from the hftp URI.
     fsUri = URI.create("hftp://localhost:"+httpPort);
-    fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
+    fs = (HftpFileSystem) FileSystem.newInstance(fsUri, conf);
     assertEquals(httpPort, fs.getCanonicalUri().getPort());
     checkTokenSelection(fs, httpPort, conf);
 
     // test with non-default port
     // Make sure it uses the port from the hftp URI.
     fsUri = URI.create("hftp://localhost:"+(httpPort+1));
-    fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
+    fs = (HftpFileSystem) FileSystem.newInstance(fsUri, conf);
     assertEquals(httpPort+1, fs.getCanonicalUri().getPort());
     checkTokenSelection(fs, httpPort + 1, conf);
 
@@ -116,7 +115,7 @@ public class TestHftpDelegationToken {
     SecurityUtilTestHelper.setTokenServiceUseIp(true);
 
     Configuration conf = new Configuration();
-    conf.setClass("fs.hsftp.impl", MyHsftpFileSystem.class, FileSystem.class);
+    conf.setClass("fs.hsftp.impl", HsftpFileSystem.class, FileSystem.class);
 
     int httpPort = 80;
     int httpsPort = 443;
@@ -125,19 +124,19 @@ public class TestHftpDelegationToken {
 
     // test with implicit default port
     URI fsUri = URI.create("hsftp://localhost");
-    MyHsftpFileSystem fs = (MyHsftpFileSystem) FileSystem.newInstance(fsUri, conf);
+    HsftpFileSystem fs = (HsftpFileSystem) FileSystem.newInstance(fsUri, conf);
     assertEquals(httpsPort, fs.getCanonicalUri().getPort());
     checkTokenSelection(fs, httpsPort, conf);
 
     // test with explicit default port
     fsUri = URI.create("hsftp://localhost:"+httpsPort);
-    fs = (MyHsftpFileSystem) FileSystem.newInstance(fsUri, conf);
+    fs = (HsftpFileSystem) FileSystem.newInstance(fsUri, conf);
     assertEquals(httpsPort, fs.getCanonicalUri().getPort());
     checkTokenSelection(fs, httpsPort, conf);
 
     // test with non-default port
     fsUri = URI.create("hsftp://localhost:"+(httpsPort+1));
-    fs = (MyHsftpFileSystem) FileSystem.newInstance(fsUri, conf);
+    fs = (HsftpFileSystem) FileSystem.newInstance(fsUri, conf);
     assertEquals(httpsPort+1, fs.getCanonicalUri().getPort());
     checkTokenSelection(fs, httpsPort+1, conf);
 
@@ -197,6 +196,9 @@ public class TestHftpDelegationToken {
     UserGroupInformation ugi =
         UserGroupInformation.createUserForTesting(fs.getUri().getAuthority(), new String[]{});
 
+    @SuppressWarnings("unchecked")
+    TokenAspect<HftpFileSystem> aspect = (TokenAspect<HftpFileSystem>) Whitebox.getInternalState(fs, "tokenAspect");
+
     // use ip-based tokens
     SecurityUtilTestHelper.setTokenServiceUseIp(true);
 
@@ -208,7 +210,7 @@ public class TestHftpDelegationToken {
     ugi.addToken(hdfsToken);
 
     // test fallback to hdfs token
-    Token<?> token = fs.selectDelegationToken(ugi);
+    Token<?> token = aspect.selectDelegationToken(ugi);
     assertNotNull(token);
     assertEquals(hdfsToken, token);
 
@@ -217,13 +219,13 @@ public class TestHftpDelegationToken {
         new byte[0], new byte[0],
         HftpFileSystem.TOKEN_KIND, new Text("127.0.0.1:"+port));
     ugi.addToken(hftpToken);
-    token = fs.selectDelegationToken(ugi);
+    token = aspect.selectDelegationToken(ugi);
     assertNotNull(token);
     assertEquals(hftpToken, token);
 
     // switch to using host-based tokens, no token should match
     SecurityUtilTestHelper.setTokenServiceUseIp(false);
-    token = fs.selectDelegationToken(ugi);
+    token = aspect.selectDelegationToken(ugi);
     assertNull(token);
 
     // test fallback to hdfs token
@@ -232,7 +234,7 @@ public class TestHftpDelegationToken {
         DelegationTokenIdentifier.HDFS_DELEGATION_KIND,
         new Text("localhost:8020"));
     ugi.addToken(hdfsToken);
-    token = fs.selectDelegationToken(ugi);
+    token = aspect.selectDelegationToken(ugi);
     assertNotNull(token);
     assertEquals(hdfsToken, token);
 
@@ -241,36 +243,8 @@ public class TestHftpDelegationToken {
         new byte[0], new byte[0],
         HftpFileSystem.TOKEN_KIND, new Text("localhost:"+port));
     ugi.addToken(hftpToken);
-    token = fs.selectDelegationToken(ugi);
+    token = aspect.selectDelegationToken(ugi);
     assertNotNull(token);
     assertEquals(hftpToken, token);
   }
-
-  static class MyHftpFileSystem extends HftpFileSystem {
-    @Override
-    public URI getCanonicalUri() {
-      return super.getCanonicalUri();
-    }
-    @Override
-    public int getDefaultPort() {
-      return super.getDefaultPort();
-    }
-    // don't automatically get a token
-    @Override
-    protected void initDelegationToken() throws IOException {}
-  }
-
-  static class MyHsftpFileSystem extends HsftpFileSystem {
-    @Override
-    public URI getCanonicalUri() {
-      return super.getCanonicalUri();
-    }
-    @Override
-    public int getDefaultPort() {
-      return super.getDefaultPort();
-    }
-    // don't automatically get a token
-    @Override
-    protected void initDelegationToken() throws IOException {}
-  }
 }

+ 298 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestTokenAspect.java

@@ -0,0 +1,298 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.web;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegationTokenRenewer;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.SecurityUtilTestHelper;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.Progressable;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.internal.util.reflection.Whitebox;
+
+public class TestTokenAspect {
+
+  private static class DummyFs extends FileSystem implements
+      DelegationTokenRenewer.Renewable, TokenAspect.TokenManagementDelegator {
+
+    private static final Text TOKEN_KIND = new Text("DummyFS Token");
+    private boolean emulateSecurityEnabled;
+    private TokenAspect<DummyFs> tokenAspect;
+    private UserGroupInformation ugi = UserGroupInformation
+        .createUserForTesting("foo", new String[] { "bar" });
+    private URI uri;
+
+    @Override
+    public FSDataOutputStream append(Path f, int bufferSize,
+        Progressable progress) throws IOException {
+      return null;
+    }
+
+    @Override
+    public void cancelDelegationToken(Token<?> token) throws IOException {
+    }
+
+    @Override
+    public FSDataOutputStream create(Path f, FsPermission permission,
+        boolean overwrite, int bufferSize, short replication, long blockSize,
+        Progressable progress) throws IOException {
+      return null;
+    }
+
+    @Override
+    public boolean delete(Path f, boolean recursive) throws IOException {
+      return false;
+    }
+
+    @Override
+    public URI getCanonicalUri() {
+      return super.getCanonicalUri();
+    }
+
+    @Override
+    public FileStatus getFileStatus(Path f) throws IOException {
+      return null;
+    }
+
+    @Override
+    public Token<?> getRenewToken() {
+      return null;
+    }
+
+    @Override
+    public URI getUri() {
+      return uri;
+    }
+
+    @Override
+    public Path getWorkingDirectory() {
+      return null;
+    }
+
+    @Override
+    public void initialize(URI name, Configuration conf) throws IOException {
+      super.initialize(name, conf);
+      setConf(conf);
+      try {
+        this.uri = new URI(name.getScheme(), name.getAuthority(), null, null,
+            null);
+      } catch (URISyntaxException e) {
+        throw new IllegalArgumentException(e);
+      }
+
+      tokenAspect = new TokenAspect<DummyFs>(this, DummyFs.TOKEN_KIND);
+      if (emulateSecurityEnabled || UserGroupInformation.isSecurityEnabled()) {
+        tokenAspect.initDelegationToken(ugi);
+      }
+    }
+
+    @Override
+    public FileStatus[] listStatus(Path f) throws FileNotFoundException,
+        IOException {
+      return null;
+    }
+
+    @Override
+    public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+      return false;
+    }
+
+    @Override
+    public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+      return null;
+    }
+
+    @Override
+    public boolean rename(Path src, Path dst) throws IOException {
+      return false;
+    }
+
+    @Override
+    public long renewDelegationToken(Token<?> token) throws IOException {
+      return 0;
+    }
+
+    @Override
+    public <T extends TokenIdentifier> void setDelegationToken(Token<T> token) {
+    }
+
+    @Override
+    public void setWorkingDirectory(Path new_dir) {
+    }
+  }
+
+  @Test
+  public void testGetRemoteToken() throws IOException, URISyntaxException {
+    Configuration conf = new Configuration();
+    UserGroupInformation.setConfiguration(conf);
+    DummyFs fs = spy(new DummyFs());
+    Token<TokenIdentifier> token = new Token<TokenIdentifier>(new byte[0],
+        new byte[0], DummyFs.TOKEN_KIND, new Text("127.0.0.1:1234"));
+
+    doReturn(token).when(fs).getDelegationToken(anyString());
+    fs.initialize(new URI("dummyfs://127.0.0.1:1234"), conf);
+
+    fs.tokenAspect.ensureTokenInitialized();
+
+    // Select a token, store and renew it
+    verify(fs).setDelegationToken(token);
+    assertNotNull(Whitebox.getInternalState(fs.tokenAspect, "dtRenewer"));
+    assertNotNull(Whitebox.getInternalState(fs.tokenAspect, "action"));
+  }
+
+  @Test
+  public void testGetRemoteTokenFailure() throws IOException,
+      URISyntaxException {
+    Configuration conf = new Configuration();
+    UserGroupInformation.setConfiguration(conf);
+    DummyFs fs = spy(new DummyFs());
+    IOException e = new IOException();
+    doThrow(e).when(fs).getDelegationToken(anyString());
+
+    fs.emulateSecurityEnabled = true;
+    fs.initialize(new URI("dummyfs://127.0.0.1:1234"), conf);
+    try {
+      fs.tokenAspect.ensureTokenInitialized();
+    } catch (IOException exc) {
+      assertEquals(e, exc);
+    }
+  }
+
+  @Test
+  public void testInitWithNoTokens() throws IOException, URISyntaxException {
+    Configuration conf = new Configuration();
+    UserGroupInformation.setConfiguration(conf);
+    DummyFs fs = spy(new DummyFs());
+    doReturn(null).when(fs).getDelegationToken(anyString());
+    fs.initialize(new URI("dummyfs://127.0.0.1:1234"), conf);
+
+    fs.tokenAspect.ensureTokenInitialized();
+
+    // No token will be selected.
+    verify(fs, never()).setDelegationToken(
+        Mockito.<Token<? extends TokenIdentifier>> any());
+  }
+
+  @Test
+  public void testInitWithUGIToken() throws IOException, URISyntaxException {
+    Configuration conf = new Configuration();
+    UserGroupInformation.setConfiguration(conf);
+    DummyFs fs = spy(new DummyFs());
+    doReturn(null).when(fs).getDelegationToken(anyString());
+
+    Token<TokenIdentifier> token = new Token<TokenIdentifier>(new byte[0],
+        new byte[0], DummyFs.TOKEN_KIND, new Text("127.0.0.1:1234"));
+    fs.ugi.addToken(token);
+    fs.ugi.addToken(new Token<TokenIdentifier>(new byte[0], new byte[0],
+        new Text("Other token"), new Text("127.0.0.1:8021")));
+    assertEquals("wrong tokens in user", 2, fs.ugi.getTokens().size());
+
+    fs.emulateSecurityEnabled = true;
+    fs.initialize(new URI("dummyfs://127.0.0.1:1234"), conf);
+    fs.tokenAspect.ensureTokenInitialized();
+
+    // Select a token from ugi (not from the remote host), store it but don't
+    // renew it
+    verify(fs).setDelegationToken(token);
+    verify(fs, never()).getDelegationToken(anyString());
+    assertNull(Whitebox.getInternalState(fs.tokenAspect, "dtRenewer"));
+    assertNull(Whitebox.getInternalState(fs.tokenAspect, "action"));
+  }
+
+  @Test
+  public void testTokenSelectionPreferences() throws IOException,
+      URISyntaxException {
+    Configuration conf = new Configuration();
+    DummyFs fs = spy(new DummyFs());
+    doReturn(null).when(fs).getDelegationToken(anyString());
+    fs.initialize(new URI("dummyfs://localhost:1234"), conf);
+    TokenAspect<DummyFs> aspect = new TokenAspect<DummyFs>(fs,
+        DummyFs.TOKEN_KIND);
+    UserGroupInformation ugi = UserGroupInformation.createUserForTesting("foo",
+        new String[] { "bar" });
+    UserGroupInformation.setConfiguration(conf);
+
+    // use ip-based tokens
+    SecurityUtilTestHelper.setTokenServiceUseIp(true);
+
+    // test fallback to hdfs token
+    Token<TokenIdentifier> hdfsToken = new Token<TokenIdentifier>(new byte[0],
+        new byte[0], DelegationTokenIdentifier.HDFS_DELEGATION_KIND, new Text(
+            "127.0.0.1:8020"));
+    ugi.addToken(hdfsToken);
+
+    // test fallback to hdfs token
+    Token<?> token = aspect.selectDelegationToken(ugi);
+    assertEquals(hdfsToken, token);
+
+    // test dummyfs is favored over hdfs
+    Token<TokenIdentifier> dummyFsToken = new Token<TokenIdentifier>(
+        new byte[0], new byte[0], DummyFs.TOKEN_KIND,
+        new Text("127.0.0.1:1234"));
+    ugi.addToken(dummyFsToken);
+    token = aspect.selectDelegationToken(ugi);
+    assertEquals(dummyFsToken, token);
+
+    // switch to using host-based tokens, no token should match
+    SecurityUtilTestHelper.setTokenServiceUseIp(false);
+    token = aspect.selectDelegationToken(ugi);
+    assertNull(token);
+
+    // test fallback to hdfs token
+    hdfsToken = new Token<TokenIdentifier>(new byte[0], new byte[0],
+        DelegationTokenIdentifier.HDFS_DELEGATION_KIND, new Text(
+            "localhost:8020"));
+    ugi.addToken(hdfsToken);
+    token = aspect.selectDelegationToken(ugi);
+    assertEquals(hdfsToken, token);
+
+    // test dummyfs is favored over hdfs
+    dummyFsToken = new Token<TokenIdentifier>(new byte[0], new byte[0],
+        DummyFs.TOKEN_KIND, new Text("localhost:1234"));
+    ugi.addToken(dummyFsToken);
+    token = aspect.selectDelegationToken(ugi);
+    assertEquals(dummyFsToken, token);
+  }
+}

+ 36 - 20
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenRemoteFetcher.java

@@ -26,6 +26,8 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.nio.charset.Charset;
 import java.util.Iterator;
 import java.util.Map;
@@ -37,10 +39,12 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
 import org.apache.hadoop.hdfs.web.HftpFileSystem;
+import org.apache.hadoop.hdfs.web.URLConnectionFactory;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.log4j.Logger;
 import org.jboss.netty.bootstrap.ServerBootstrap;
@@ -59,6 +63,7 @@ import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
 import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
 import org.jboss.netty.handler.codec.http.HttpChunkAggregator;
 import org.jboss.netty.handler.codec.http.HttpHeaders;
+import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.jboss.netty.handler.codec.http.HttpRequest;
 import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
 import org.jboss.netty.handler.codec.http.HttpResponse;
@@ -78,9 +83,10 @@ public class TestDelegationTokenRemoteFetcher {
 
   private static final String EXP_DATE = "124123512361236";
   private static final String tokenFile = "http.file.dta";
+  private static final URLConnectionFactory connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
 
   private int httpPort;
-  private String serviceUrl;
+  private URI serviceUrl;
   private FileSystem fileSys;
   private Configuration conf;
   private ServerBootstrap bootstrap;
@@ -92,7 +98,7 @@ public class TestDelegationTokenRemoteFetcher {
     conf = new Configuration();
     fileSys = FileSystem.getLocal(conf);
     httpPort = NetUtils.getFreeSocketPort();
-    serviceUrl = "http://localhost:" + httpPort;
+    serviceUrl = new URI("http://localhost:" + httpPort);
     testToken = createToken(serviceUrl);
   }
 
@@ -121,9 +127,9 @@ public class TestDelegationTokenRemoteFetcher {
    * try to fetch token without http server with IOException
    */
   @Test
-  public void testTokenRenewFail() {
+  public void testTokenRenewFail() throws AuthenticationException {
     try {
-      DelegationTokenFetcher.renewDelegationToken(serviceUrl, testToken);
+      DelegationTokenFetcher.renewDelegationToken(connectionFactory, serviceUrl, testToken);
       fail("Token fetcher shouldn't be able to renew tokens in absense of NN");
     } catch (IOException ex) {
     } 
@@ -133,9 +139,9 @@ public class TestDelegationTokenRemoteFetcher {
    * try cancel token without http server with IOException
    */
   @Test
-  public void expectedTokenCancelFail() {
+  public void expectedTokenCancelFail() throws AuthenticationException {
     try {
-      DelegationTokenFetcher.cancelDelegationToken(serviceUrl, testToken);
+      DelegationTokenFetcher.cancelDelegationToken(connectionFactory, serviceUrl, testToken);
       fail("Token fetcher shouldn't be able to cancel tokens in absense of NN");
     } catch (IOException ex) {
     } 
@@ -145,11 +151,12 @@ public class TestDelegationTokenRemoteFetcher {
    * try fetch token and get http response with error
    */
   @Test  
-  public void expectedTokenRenewErrorHttpResponse() {
+  public void expectedTokenRenewErrorHttpResponse()
+      throws AuthenticationException, URISyntaxException {
     bootstrap = startHttpServer(httpPort, testToken, serviceUrl);
     try {
-      DelegationTokenFetcher.renewDelegationToken(serviceUrl + "/exception", 
-          createToken(serviceUrl));
+      DelegationTokenFetcher.renewDelegationToken(connectionFactory, new URI(
+          serviceUrl.toString() + "/exception"), createToken(serviceUrl));
       fail("Token fetcher shouldn't be able to renew tokens using an invalid"
           + " NN URL");
     } catch (IOException ex) {
@@ -159,13 +166,14 @@ public class TestDelegationTokenRemoteFetcher {
   }
   
   /**
-   *   
    *
    */
   @Test
-  public void testCancelTokenFromHttp() throws IOException {
+  public void testCancelTokenFromHttp() throws IOException,
+      AuthenticationException {
     bootstrap = startHttpServer(httpPort, testToken, serviceUrl);
-    DelegationTokenFetcher.cancelDelegationToken(serviceUrl, testToken);
+    DelegationTokenFetcher.cancelDelegationToken(connectionFactory, serviceUrl,
+        testToken);
     if (assertionError != null)
       throw assertionError;
   }
@@ -174,11 +182,12 @@ public class TestDelegationTokenRemoteFetcher {
    * Call renew token using http server return new expiration time
    */
   @Test
-  public void testRenewTokenFromHttp() throws IOException {
+  public void testRenewTokenFromHttp() throws IOException,
+      NumberFormatException, AuthenticationException {
     bootstrap = startHttpServer(httpPort, testToken, serviceUrl);
     assertTrue("testRenewTokenFromHttp error",
         Long.valueOf(EXP_DATE) == DelegationTokenFetcher.renewDelegationToken(
-            serviceUrl, testToken));
+            connectionFactory, serviceUrl, testToken));
     if (assertionError != null)
       throw assertionError;
   }
@@ -204,11 +213,11 @@ public class TestDelegationTokenRemoteFetcher {
       throw assertionError;
   }
   
-  private static Token<DelegationTokenIdentifier> createToken(String serviceUri) {
+  private static Token<DelegationTokenIdentifier> createToken(URI serviceUri) {
     byte[] pw = "hadoop".getBytes();
     byte[] ident = new DelegationTokenIdentifier(new Text("owner"), new Text(
         "renewer"), new Text("realuser")).getBytes();
-    Text service = new Text(serviceUri);
+    Text service = new Text(serviceUri.toString());
     return new Token<DelegationTokenIdentifier>(ident, pw,
         HftpFileSystem.TOKEN_KIND, service);
   }
@@ -301,8 +310,15 @@ public class TestDelegationTokenRemoteFetcher {
     public void messageReceived(ChannelHandlerContext ctx, final MessageEvent e)
         throws Exception {
       HttpRequest request = (HttpRequest) e.getMessage();
-      if (request.getMethod() != GET) {
-        return;
+
+      if (request.getMethod() == HttpMethod.OPTIONS) {
+        // Mimic SPNEGO authentication
+        HttpResponse response = new DefaultHttpResponse(HTTP_1_1,
+            HttpResponseStatus.OK);
+        response.addHeader("Set-Cookie", "hadoop-auth=1234");
+        e.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
+      } else if (request.getMethod() != GET) {
+        e.getChannel().close();
       }
       UnmodifiableIterator<Map.Entry<String, Handler>> iter = routes.entrySet()
           .iterator();
@@ -338,7 +354,7 @@ public class TestDelegationTokenRemoteFetcher {
   }
 
   private ServerBootstrap startHttpServer(int port,
-      final Token<DelegationTokenIdentifier> token, final String url) {
+      final Token<DelegationTokenIdentifier> token, final URI url) {
     ServerBootstrap bootstrap = new ServerBootstrap(
         new NioServerSocketChannelFactory(Executors.newCachedThreadPool(),
             Executors.newCachedThreadPool()));
@@ -348,7 +364,7 @@ public class TestDelegationTokenRemoteFetcher {
       public ChannelPipeline getPipeline() throws Exception {
         return Channels.pipeline(new HttpRequestDecoder(),
             new HttpChunkAggregator(65536), new HttpResponseEncoder(),
-            new CredentialsLogicHandler(token, url));
+            new CredentialsLogicHandler(token, url.toString()));
       }
     });
     bootstrap.bind(new InetSocketAddress("localhost", port));

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

@@ -170,6 +170,14 @@ Release 2.3.0 - UNRELEASED
     MAPREDUCE-5613. DefaultSpeculator holds and checks hashmap that is always
     empty (Gera Shegalov via Sandy Ryza)
 
+    MAPREDUCE-5431. Missing pom dependency in MR-client (Timothy St. Clair
+    via stevel)
+
+    MAPREDUCE-5624 Move grizzly-test and junit dependencies to test scope
+    (Ted Yu via stevel)
+
+    MAPREDUCE-5481. Enable uber jobs to have multiple reducers (Sandy Ryza)
+
   OPTIMIZATIONS
 
     MAPREDUCE-5484. YarnChild unnecessarily loads job conf twice (Sandy Ryza)
@@ -209,6 +217,9 @@ Release 2.3.0 - UNRELEASED
 
     MAPREDUCE-5610. TestSleepJob fails in jdk7 (Jonathan Eagles via jlowe)
 
+    MAPREDUCE-5616. MR Client-AppMaster RPC max retries on socket timeout is too
+    high. (cnauroth)
+
 Release 2.2.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 88 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java

@@ -20,7 +20,9 @@ package org.apache.hadoop.mapred;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 
@@ -35,6 +37,7 @@ import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobCounter;
 import org.apache.hadoop.mapreduce.MRConfig;
+import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
@@ -168,6 +171,10 @@ public class LocalContainerLauncher extends AbstractService implements
     public void run() {
       ContainerLauncherEvent event = null;
 
+      // Collect locations of map outputs to give to reduces
+      Map<TaskAttemptID, MapOutputFile> localMapFiles =
+          new HashMap<TaskAttemptID, MapOutputFile>();
+      
       // _must_ either run subtasks sequentially or accept expense of new JVMs
       // (i.e., fork()), else will get weird failures when maps try to create/
       // write same dirname or filename:  no chdir() in Java
@@ -223,7 +230,7 @@ public class LocalContainerLauncher extends AbstractService implements
               context.getEventHandler().handle(jce);
             }
             runSubtask(remoteTask, ytask.getType(), attemptID, numMapTasks,
-                       (numReduceTasks > 0));
+                       (numReduceTasks > 0), localMapFiles);
             
           } catch (RuntimeException re) {
             JobCounterUpdateEvent jce = new JobCounterUpdateEvent(attemptID.getTaskId().getJobId());
@@ -265,7 +272,8 @@ public class LocalContainerLauncher extends AbstractService implements
                             final TaskType taskType,
                             TaskAttemptId attemptID,
                             final int numMapTasks,
-                            boolean renameOutputs)
+                            boolean renameOutputs,
+                            Map<TaskAttemptID, MapOutputFile> localMapFiles)
     throws RuntimeException, IOException {
       org.apache.hadoop.mapred.TaskAttemptID classicAttemptID =
           TypeConverter.fromYarn(attemptID);
@@ -309,7 +317,9 @@ public class LocalContainerLauncher extends AbstractService implements
           map.run(conf, umbilical);
 
           if (renameOutputs) {
-            renameMapOutputForReduce(conf, attemptID, map.getMapOutputFile());
+            MapOutputFile renamed = renameMapOutputForReduce(conf, attemptID,
+                map.getMapOutputFile());
+            localMapFiles.put(classicAttemptID, renamed);
           }
           relocalize();
 
@@ -335,10 +345,11 @@ public class LocalContainerLauncher extends AbstractService implements
           conf.set(MRConfig.MASTER_ADDRESS, "local");  // bypass shuffle
 
           ReduceTask reduce = (ReduceTask)task;
+          reduce.setLocalMapFiles(localMapFiles);
           reduce.setConf(conf);          
 
           reduce.run(conf, umbilical);
-          //relocalize();  // needed only if more than one reducer supported (is MAPREDUCE-434 fixed yet?)
+          relocalize();
         }
 
       } catch (FSError e) {
@@ -387,15 +398,16 @@ public class LocalContainerLauncher extends AbstractService implements
      * so there are no particular compatibility issues.)
      */
     @SuppressWarnings("deprecation")
-    private void renameMapOutputForReduce(JobConf conf, TaskAttemptId mapId,
-                                          MapOutputFile subMapOutputFile)
-    throws IOException {
+    private MapOutputFile renameMapOutputForReduce(JobConf conf,
+        TaskAttemptId mapId, MapOutputFile subMapOutputFile) throws IOException {
       FileSystem localFs = FileSystem.getLocal(conf);
       // move map output to reduce input
       Path mapOut = subMapOutputFile.getOutputFile();
       FileStatus mStatus = localFs.getFileStatus(mapOut);      
       Path reduceIn = subMapOutputFile.getInputFileForWrite(
           TypeConverter.fromYarn(mapId).getTaskID(), mStatus.getLen());
+      Path mapOutIndex = new Path(mapOut.toString() + ".index");
+      Path reduceInIndex = new Path(reduceIn.toString() + ".index");
       if (LOG.isDebugEnabled()) {
         LOG.debug("Renaming map output file for task attempt "
             + mapId.toString() + " from original location " + mapOut.toString()
@@ -407,6 +419,10 @@ public class LocalContainerLauncher extends AbstractService implements
       }
       if (!localFs.rename(mapOut, reduceIn))
         throw new IOException("Couldn't rename " + mapOut);
+      if (!localFs.rename(mapOutIndex, reduceInIndex))
+        throw new IOException("Couldn't rename " + mapOutIndex);
+      
+      return new RenamedMapOutputFile(reduceIn);
     }
 
     /**
@@ -441,5 +457,70 @@ public class LocalContainerLauncher extends AbstractService implements
     }
 
   } // end SubtaskRunner
+  
+  private static class RenamedMapOutputFile extends MapOutputFile {
+    private Path path;
+    
+    public RenamedMapOutputFile(Path path) {
+      this.path = path;
+    }
+    
+    @Override
+    public Path getOutputFile() throws IOException {
+      return path;
+    }
+
+    @Override
+    public Path getOutputFileForWrite(long size) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public Path getOutputFileForWriteInVolume(Path existing) {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public Path getOutputIndexFile() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public Path getOutputIndexFileForWrite(long size) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public Path getOutputIndexFileForWriteInVolume(Path existing) {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public Path getSpillFile(int spillNumber) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public Path getSpillFileForWrite(int spillNumber, long size)
+        throws IOException {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public Path getSpillIndexFile(int spillNumber) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public Path getSpillIndexFileForWrite(int spillNumber, long size)
+        throws IOException {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public Path getInputFile(int mapId) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public Path getInputFileForWrite(TaskID mapId, long size)
+        throws IOException {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public void removeAll() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+  }
 
 }

+ 1 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java

@@ -1173,11 +1173,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
     // these are no longer "system" settings, necessarily; user may override
     int sysMaxMaps = conf.getInt(MRJobConfig.JOB_UBERTASK_MAXMAPS, 9);
 
-    //FIXME: handling multiple reduces within a single AM does not seem to
-    //work.
     int sysMaxReduces = conf.getInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 1);
-    boolean isValidUberMaxReduces = (sysMaxReduces == 0)
-        || (sysMaxReduces == 1);
 
     long sysMaxBytes = conf.getLong(MRJobConfig.JOB_UBERTASK_MAXBYTES,
         fs.getDefaultBlockSize(this.remoteJobSubmitDir)); // FIXME: this is wrong; get FS from
@@ -1225,7 +1221,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
     // and thus requires sequential execution.
     isUber = uberEnabled && smallNumMapTasks && smallNumReduceTasks
         && smallInput && smallMemory && smallCpu 
-        && notChainJob && isValidUberMaxReduces;
+        && notChainJob;
 
     if (isUber) {
       LOG.info("Uberizing job " + jobId + ": " + numMapTasks + "m+"
@@ -1259,8 +1255,6 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
         msg.append(" too much RAM;");
       if (!notChainJob)
         msg.append(" chainjob;");
-      if (!isValidUberMaxReduces)
-        msg.append(" not supported uber max reduces");
       LOG.info(msg.toString());
     }
   }

+ 9 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java

@@ -357,13 +357,21 @@ public interface MRJobConfig {
 
   public static final String MR_AM_PREFIX = MR_PREFIX + "am.";
 
-  /** The number of client retires to the AM - before reconnecting to the RM
+  /** The number of client retries to the AM - before reconnecting to the RM
    * to fetch Application State. 
    */
   public static final String MR_CLIENT_TO_AM_IPC_MAX_RETRIES = 
     MR_PREFIX + "client-am.ipc.max-retries";
   public static final int DEFAULT_MR_CLIENT_TO_AM_IPC_MAX_RETRIES = 3;
   
+  /** The number of client retries on socket timeouts to the AM - before
+   * reconnecting to the RM to fetch Application Status.
+   */
+  public static final String MR_CLIENT_TO_AM_IPC_MAX_RETRIES_ON_TIMEOUTS =
+    MR_PREFIX + "yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts";
+  public static final int
+    DEFAULT_MR_CLIENT_TO_AM_IPC_MAX_RETRIES_ON_TIMEOUTS = 3;
+
   /**
    * The number of client retries to the RM/HS before throwing exception.
    */

+ 7 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml

@@ -1010,6 +1010,13 @@
     to the RM to fetch Application Status.</description>
 </property>
 
+<property>
+  <name>yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts</name>
+  <value>3</value>
+  <description>The number of client retries on socket timeouts to the AM - before
+    reconnecting to the RM to fetch Application Status.</description>
+</property>
+
 <property>
   <name>yarn.app.mapreduce.client.max-retries</name>
   <value>3</value>

+ 4 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java

@@ -107,6 +107,10 @@ public class ClientServiceDelegate {
         CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
         this.conf.getInt(MRJobConfig.MR_CLIENT_TO_AM_IPC_MAX_RETRIES,
             MRJobConfig.DEFAULT_MR_CLIENT_TO_AM_IPC_MAX_RETRIES));
+    this.conf.setInt(
+        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
+        this.conf.getInt(MRJobConfig.MR_CLIENT_TO_AM_IPC_MAX_RETRIES_ON_TIMEOUTS,
+            MRJobConfig.DEFAULT_MR_CLIENT_TO_AM_IPC_MAX_RETRIES_ON_TIMEOUTS));
     this.rm = rm;
     this.jobId = jobId;
     this.historyServerProxy = historyServerProxy;

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

@@ -20,6 +20,7 @@ package org.apache.hadoop.mapreduce.v2;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -39,12 +40,14 @@ import org.junit.Test;
 public class TestUberAM extends TestMRJobs {
 
   private static final Log LOG = LogFactory.getLog(TestUberAM.class);
-
+  private int numSleepReducers;
+  
   @BeforeClass
   public static void setup() throws IOException {
     TestMRJobs.setup();
     if (mrCluster != null) {
     	mrCluster.getConfig().setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, true);
+    	mrCluster.getConfig().setInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 3);
     }
   }
 
@@ -52,8 +55,19 @@ public class TestUberAM extends TestMRJobs {
   @Test
   public void testSleepJob()
   throws IOException, InterruptedException, ClassNotFoundException {
+    numSleepReducers = 1;
     if (mrCluster != null) {
-    	mrCluster.getConfig().setInt("TestMRJobs.testSleepJob.reduces", 1);
+    	mrCluster.getConfig().setInt("TestMRJobs.testSleepJob.reduces", numSleepReducers);
+    }
+    super.testSleepJob();
+  }
+  
+  @Test
+  public void testSleepJobWithMultipleReducers()
+  throws IOException, InterruptedException, ClassNotFoundException {
+    numSleepReducers = 3;
+    if (mrCluster != null) {
+      mrCluster.getConfig().setInt("TestMRJobs.testSleepJob.reduces", numSleepReducers);
     }
     super.testSleepJob();
   }
@@ -67,7 +81,7 @@ public class TestUberAM extends TestMRJobs {
         .getValue());
     Assert.assertEquals(3, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
         .getValue());
-    Assert.assertEquals(1,
+    Assert.assertEquals(numSleepReducers,
         counters.findCounter(JobCounter.TOTAL_LAUNCHED_REDUCES).getValue());
     Assert
         .assertTrue(counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS) != null
@@ -76,11 +90,11 @@ public class TestUberAM extends TestMRJobs {
         .assertTrue(counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS) != null
             && counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS).getValue() != 0);
 
-    Assert.assertEquals(3, counters.findCounter(JobCounter.NUM_UBER_SUBMAPS)
-        .getValue());
-    Assert.assertEquals(1, counters.findCounter(JobCounter.NUM_UBER_SUBREDUCES)
-        .getValue());
-    Assert.assertEquals(4,
+    Assert.assertEquals(3,
+        counters.findCounter(JobCounter.NUM_UBER_SUBMAPS).getValue());
+    Assert.assertEquals(numSleepReducers,
+        counters.findCounter(JobCounter.NUM_UBER_SUBREDUCES).getValue());
+    Assert.assertEquals(3 + numSleepReducers,
         counters.findCounter(JobCounter.TOTAL_LAUNCHED_UBERTASKS).getValue());
   }
 
@@ -138,8 +152,10 @@ public class TestUberAM extends TestMRJobs {
 
     TaskCompletionEvent[] events = job.getTaskCompletionEvents(0, 2);
     Assert.assertEquals(1, events.length);
-    Assert.assertEquals(TaskCompletionEvent.Status.TIPFAILED,
-        events[0].getStatus());
+    // TIPFAILED if it comes from the AM, FAILED if it comes from the JHS
+    TaskCompletionEvent.Status status = events[0].getStatus();
+    Assert.assertTrue(status == TaskCompletionEvent.Status.FAILED ||
+        status == TaskCompletionEvent.Status.TIPFAILED);
     Assert.assertEquals(JobStatus.State.FAILED, job.getJobState());
     
     //Disabling till UberAM honors MRJobConfig.MAP_MAX_ATTEMPTS

+ 10 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml

@@ -158,6 +158,16 @@
       <artifactId>commons-lang</artifactId>
       <scope>provided</scope>
     </dependency>
+    <dependency>
+      <groupId>commons-collections</groupId>
+      <artifactId>commons-collections</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.sun.jersey.jersey-test-framework</groupId>
+      <artifactId>jersey-test-framework-grizzly2</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
  
   <build>

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml

@@ -93,6 +93,11 @@
        <type>test-jar</type>
        <scope>test</scope>
      </dependency>
+    <dependency>
+      <groupId>com.sun.jersey.jersey-test-framework</groupId>
+      <artifactId>jersey-test-framework-grizzly2</artifactId>
+      <scope>test</scope>
+    </dependency>
      <dependency>
        <groupId>org.apache.hadoop</groupId>
        <artifactId>hadoop-mapreduce-client-hs</artifactId>

+ 7 - 2
hadoop-project/pom.xml

@@ -324,8 +324,8 @@
       </dependency>
       <dependency>
         <groupId>org.apache.commons</groupId>
-        <artifactId>commons-math</artifactId>
-        <version>2.1</version>
+        <artifactId>commons-math3</artifactId>
+        <version>3.1.1</version>
       </dependency>
       <dependency>
         <groupId>org.apache.commons</groupId>
@@ -745,6 +745,11 @@
         <artifactId>hadoop-sls</artifactId>
         <version>${project.version}</version>
       </dependency>
+      <dependency>
+        <groupId>com.google.code.findbugs</groupId>
+        <artifactId>jsr305</artifactId>
+        <version>1.3.9</version>
+      </dependency>    
     </dependencies>
   </dependencyManagement>
 

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

@@ -35,6 +35,11 @@
   </properties>
 
   <dependencies>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-annotations</artifactId>

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

@@ -32,6 +32,11 @@
   </properties>
 
   <dependencies>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-annotations</artifactId>

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

@@ -37,6 +37,11 @@
   </properties>
 
   <dependencies>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>

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

@@ -35,6 +35,11 @@
   </properties>
 
   <dependencies>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-annotations</artifactId>

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

@@ -32,6 +32,11 @@
   </properties>
 
   <dependencies>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-annotations</artifactId>

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

@@ -35,6 +35,11 @@
   </properties>
 
   <dependencies>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-annotations</artifactId>

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

@@ -36,6 +36,11 @@
   </properties>
 
   <dependencies>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-annotations</artifactId>

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

@@ -40,6 +40,9 @@ Release 2.3.0 - UNRELEASED
     YARN-311. RM/scheduler support for dynamic resource configuration.
     (Junping Du via llu)
 
+    YARN-1392. Allow sophisticated app-to-queue placement policies in the Fair
+    Scheduler (Sandy Ryza)
+
   IMPROVEMENTS
 
     YARN-7. Support CPU resource for DistributedShell. (Junping Du via llu)
@@ -94,6 +97,9 @@ Release 2.3.0 - UNRELEASED
     YARN-1387. RMWebServices should use ClientRMService for filtering
     applications (Karthik Kambatla via Sandy Ryza)
 
+    YARN-1222. Make improvements in ZKRMStateStore for fencing (Karthik
+    Kambatla via bikas)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -127,6 +133,9 @@ Release 2.3.0 - UNRELEASED
     YARN-1400. yarn.cmd uses HADOOP_RESOURCEMANAGER_OPTS. Should be
     YARN_RESOURCEMANAGER_OPTS. (Raja Aluri via cnauroth)
 
+    YARN-1401. With zero sleep-delay-before-sigkill.ms, no signal is ever sent
+    (Gera Shegalov via Sandy Ryza)
+
 Release 2.2.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 6 - 0
hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml

@@ -178,6 +178,12 @@
     <Field name="minimumAllocation" />
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>
+  <!-- Inconsistent sync warning - numRetries is only initialized once and never changed -->
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore" />
+    <Field name="numRetries" />
+    <Bug pattern="IS2_INCONSISTENT_SYNC" />
+  </Match>
   <Match>
     <Class name="org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer"/>
     <Field name="renewalTimer" />

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/HAUtil.java

@@ -193,8 +193,8 @@ public class HAUtil {
     return addSuffix(prefix, getRMHAId(conf));
   }
 
-  private static String getConfValueForRMInstance(String prefix,
-                                                  Configuration conf) {
+  public static String getConfValueForRMInstance(String prefix,
+                                                 Configuration conf) {
     String confKey = getConfKeyForRMInstance(prefix, conf);
     String retVal = conf.getTrimmed(confKey);
     if (LOG.isTraceEnabled()) {
@@ -205,8 +205,8 @@ public class HAUtil {
     return retVal;
   }
 
-  static String getConfValueForRMInstance(String prefix, String defaultValue,
-                                          Configuration conf) {
+  public static String getConfValueForRMInstance(
+      String prefix, String defaultValue, Configuration conf) {
     String value = getConfValueForRMInstance(prefix, conf);
     return (value == null) ? defaultValue : value;
   }

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -328,6 +328,8 @@ public class YarnConfiguration extends Configuration {
       ZK_STATE_STORE_PREFIX + "acl";
   public static final String DEFAULT_ZK_RM_STATE_STORE_ACL =
       "world:anyone:rwcda";
+  public static final String ZK_RM_STATE_STORE_ROOT_NODE_ACL =
+      ZK_STATE_STORE_PREFIX + "root-node.acl";
 
   /** The maximum number of completed applications RM keeps. */ 
   public static final String RM_MAX_COMPLETED_APPLICATIONS =

+ 30 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -279,7 +279,11 @@
     <description>Host:Port of the ZooKeeper server where RM state will 
     be stored. This must be supplied when using
     org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
-    as the value for yarn.resourcemanager.store.class</description>
+    as the value for yarn.resourcemanager.store.class. ZKRMStateStore
+    is implicitly fenced, meaning a single ResourceManager is
+    able to use the store at any point in time. More details on this, along
+    with setting up appropriate ACLs is discussed under the description for
+    yarn.resourcemanager.zk.state-store.root-node.acl.</description>
     <name>yarn.resourcemanager.zk.state-store.address</name>
     <!--value>127.0.0.1:2181</value-->
   </property>
@@ -320,6 +324,31 @@
     <value>world:anyone:rwcda</value>
   </property>
 
+  <property>
+    <description>
+      ACLs to be used for the root znode when using ZKRMStateStore in a HA
+      scenario for fencing.
+
+      ZKRMStateStore supports implicit fencing to allow a single
+      ResourceManager write-access to the store. For fencing, the
+      ResourceManagers in the cluster share read-write-admin privileges on the
+      root node, but the Active ResourceManager claims exclusive create-delete
+      permissions.
+
+      By default, when this property is not set, we use the ACLs from
+      yarn.resourcemanager.zk.state-store.acl for shared admin access and
+      rm-address:cluster-timestamp for username-based exclusive create-delete
+      access.
+
+      This property allows users to set ACLs of their choice instead of using
+      the default mechanism. For fencing to work, the ACLs should be
+      carefully set differently on each ResourceManger such that all the
+      ResourceManagers have shared admin access and the Active ResourceManger
+      takes over (exclusively) the create-delete access.
+    </description>
+    <name>yarn.resourcemanager.zk.state-store.root-node.acl</name>
+  </property>
+
   <property>
     <description>URI pointing to the location of the FileSystem path where
     RM state will be stored. This must be supplied when using

+ 12 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java

@@ -375,13 +375,19 @@ public class ContainerLaunch implements Callable<Integer> {
         LOG.debug("Sending signal to pid " + processId
             + " as user " + user
             + " for container " + containerIdStr);
+
+        final Signal signal = sleepDelayBeforeSigKill > 0
+          ? Signal.TERM
+          : Signal.KILL;
+
+        boolean result = exec.signalContainer(user, processId, signal);
+
+        LOG.debug("Sent signal " + signal + " to pid " + processId
+          + " as user " + user
+          + " for container " + containerIdStr
+          + ", result=" + (result? "success" : "failed"));
+
         if (sleepDelayBeforeSigKill > 0) {
-          boolean result = exec.signalContainer(user,
-              processId, Signal.TERM);
-          LOG.debug("Sent signal to pid " + processId
-              + " as user " + user
-              + " for container " + containerIdStr
-              + ", result=" + (result? "success" : "failed"));
           new DelayedProcessKiller(container, user,
               processId, sleepDelayBeforeSigKill, Signal.KILL, exec).start();
         }

+ 16 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java

@@ -97,7 +97,6 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     conf.setClass(
         YarnConfiguration.NM_CONTAINER_MON_RESOURCE_CALCULATOR,
         LinuxResourceCalculatorPlugin.class, ResourceCalculatorPlugin.class);
-    conf.setLong(YarnConfiguration.NM_SLEEP_DELAY_BEFORE_SIGKILL_MS, 1000);
     super.setup();
   }
 
@@ -590,8 +589,9 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
         AuxiliaryServiceHelper.getServiceDataFromEnv(serviceName, env));
   }
 
-  @Test
-  public void testDelayedKill() throws Exception {
+  private void internalKillTest(boolean delayed) throws Exception {
+    conf.setLong(YarnConfiguration.NM_SLEEP_DELAY_BEFORE_SIGKILL_MS,
+      delayed ? 1000 : 0);
     containerManager.start();
 
     // ////// Construct the Container-id
@@ -675,7 +675,8 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     BaseContainerManagerTest.waitForContainerState(containerManager, cId,
         ContainerState.COMPLETE);
 
-    // container stop sends a sigterm followed by a sigkill
+    // if delayed container stop sends a sigterm followed by a sigkill
+    // otherwise sigkill is sent immediately 
     GetContainerStatusesRequest gcsRequest =
         GetContainerStatusesRequest.newInstance(containerIds);
     
@@ -690,7 +691,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     // Windows, because the process is not notified when killed by winutils.
     // There is no way for the process to trap and respond.  Instead, we can
     // verify that the job object with ID matching container ID no longer exists.
-    if (Shell.WINDOWS) {
+    if (Shell.WINDOWS || !delayed) {
       Assert.assertFalse("Process is still alive!",
         DefaultContainerExecutor.containerIsAlive(cId.toString()));
     } else {
@@ -713,6 +714,16 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     }
   }
 
+  @Test
+  public void testDelayedKill() throws Exception {
+    internalKillTest(true);
+  }
+
+  @Test
+  public void testImmediateKill() throws Exception {
+    internalKillTest(false);
+  }
+
   @SuppressWarnings("rawtypes")
   @Test
   public void testCallFailureWithNullLocalizedResources() {

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

@@ -67,7 +67,9 @@ public class RMHAProtocolService extends AbstractService implements
   protected HAServiceState haState = HAServiceState.INITIALIZING;
   private AccessControlList adminAcl;
   private Server haAdminServer;
-  private boolean haEnabled;
+
+  @InterfaceAudience.Private
+  boolean haEnabled;
 
   public RMHAProtocolService(ResourceManager resourceManager)  {
     super("RMHAProtocolService");
@@ -174,7 +176,8 @@ public class RMHAProtocolService extends AbstractService implements
     }
   }
 
-  private synchronized void transitionToActive() throws Exception {
+  @InterfaceAudience.Private
+  synchronized void transitionToActive() throws Exception {
     if (haState == HAServiceState.ACTIVE) {
       LOG.info("Already in active state");
       return;
@@ -205,7 +208,8 @@ public class RMHAProtocolService extends AbstractService implements
     }
   }
 
-  private synchronized void transitionToStandby(boolean initialize)
+  @InterfaceAudience.Private
+  synchronized void transitionToStandby(boolean initialize)
       throws Exception {
     if (haState == HAServiceState.STANDBY) {
       LOG.info("Already in standby state");

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

@@ -56,6 +56,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreFactory;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreOperationFailedEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreOperationFailedEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
@@ -163,6 +165,10 @@ public class ResourceManager extends CompositeService implements Recoverable {
   public ResourceManager() {
     super("ResourceManager");
   }
+
+  public RMHAProtocolService getHAService() {
+    return this.haService;
+  }
   
   public RMContext getRMContext() {
     return this.rmContext;
@@ -216,6 +222,11 @@ public class ResourceManager extends CompositeService implements Recoverable {
     return new SchedulerEventDispatcher(this.scheduler);
   }
 
+  protected RMStateStoreOperationFailedEventDispatcher
+  createRMStateStoreOperationFailedEventDispatcher() {
+    return new RMStateStoreOperationFailedEventDispatcher(haService);
+  }
+
   protected Dispatcher createDispatcher() {
     return new AsyncDispatcher();
   }
@@ -339,6 +350,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
       try {
         rmStore.init(conf);
         rmStore.setRMDispatcher(rmDispatcher);
+        rmDispatcher.register(RMStateStoreOperationFailedEventType.class,
+            createRMStateStoreOperationFailedEventDispatcher());
       } catch (Exception e) {
         // the Exception from stateStore.init() needs to be handled for
         // HA and we need to give up master status if we got fenced
@@ -632,6 +645,46 @@ public class ResourceManager extends CompositeService implements Recoverable {
     }
   }
 
+  @Private
+  public static class RMStateStoreOperationFailedEventDispatcher implements
+      EventHandler<RMStateStoreOperationFailedEvent> {
+    private final RMHAProtocolService haService;
+
+    public RMStateStoreOperationFailedEventDispatcher(
+        RMHAProtocolService haService) {
+      this.haService = haService;
+    }
+
+    @Override
+    public void handle(RMStateStoreOperationFailedEvent event) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Received a " +
+            RMStateStoreOperationFailedEvent.class.getName() + " of type " +
+            event.getType().name());
+      }
+      if (event.getType() == RMStateStoreOperationFailedEventType.FENCED) {
+        LOG.info("RMStateStore has been fenced");
+        synchronized(haService) {
+          if (haService.haEnabled) {
+            try {
+              // Transition to standby and reinit active services
+              LOG.info("Transitioning RM to Standby mode");
+              haService.transitionToStandby(true);
+              return;
+            } catch (Exception e) {
+              LOG.error("Failed to transition RM to Standby mode.");
+            }
+          }
+        }
+      }
+
+      LOG.error("Shutting down RM on receiving a " +
+          RMStateStoreOperationFailedEvent.class.getName() + " of type " +
+          event.getType().name());
+      ExitUtil.terminate(1, event.getCause());
+    }
+  }
+
   @Private
   public static final class ApplicationEventDispatcher implements
       EventHandler<RMAppEvent> {

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

@@ -43,6 +43,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
+
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -388,9 +389,13 @@ public abstract class RMStateStore extends AbstractService {
    */
   public synchronized void storeRMDelegationTokenAndSequenceNumber(
       RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate,
-      int latestSequenceNumber) throws Exception {
-    storeRMDelegationTokenAndSequenceNumberState(rmDTIdentifier, renewDate,
-      latestSequenceNumber);
+      int latestSequenceNumber) {
+    try {
+      storeRMDelegationTokenAndSequenceNumberState(rmDTIdentifier, renewDate,
+          latestSequenceNumber);
+    } catch (Exception e) {
+      notifyStoreOperationFailed(e);
+    }
   }
 
   /**
@@ -406,9 +411,12 @@ public abstract class RMStateStore extends AbstractService {
    * RMDTSecretManager call this to remove the state of a delegation token
    */
   public synchronized void removeRMDelegationToken(
-      RMDelegationTokenIdentifier rmDTIdentifier, int sequenceNumber)
-      throws Exception {
-    removeRMDelegationTokenState(rmDTIdentifier);
+      RMDelegationTokenIdentifier rmDTIdentifier, int sequenceNumber) {
+    try {
+      removeRMDelegationTokenState(rmDTIdentifier);
+    } catch (Exception e) {
+      notifyStoreOperationFailed(e);
+    }
   }
 
   /**
@@ -421,9 +429,12 @@ public abstract class RMStateStore extends AbstractService {
   /**
    * RMDTSecretManager call this to store the state of a master key
    */
-  public synchronized void storeRMDTMasterKey(DelegationKey delegationKey)
-      throws Exception {
-    storeRMDTMasterKeyState(delegationKey);
+  public synchronized void storeRMDTMasterKey(DelegationKey delegationKey) {
+    try {
+      storeRMDTMasterKeyState(delegationKey);
+    } catch (Exception e) {
+      notifyStoreOperationFailed(e);
+    }
   }
 
   /**
@@ -437,9 +448,12 @@ public abstract class RMStateStore extends AbstractService {
   /**
    * RMDTSecretManager call this to remove the state of a master key
    */
-  public synchronized void removeRMDTMasterKey(DelegationKey delegationKey)
-      throws Exception {
-    removeRMDTMasterKeyState(delegationKey);
+  public synchronized void removeRMDTMasterKey(DelegationKey delegationKey) {
+    try {
+      removeRMDTMasterKeyState(delegationKey);
+    } catch (Exception e) {
+      notifyStoreOperationFailed(e);
+    }
   }
 
   /**
@@ -539,19 +553,15 @@ public abstract class RMStateStore extends AbstractService {
       try {
         if (event.getType().equals(RMStateStoreEventType.STORE_APP)) {
           storeApplicationStateInternal(appId.toString(), appStateData);
+          notifyDoneStoringApplication(appId, storedException);
         } else {
           assert event.getType().equals(RMStateStoreEventType.UPDATE_APP);
           updateApplicationStateInternal(appId.toString(), appStateData);
+          notifyDoneUpdatingApplication(appId, storedException);
         }
       } catch (Exception e) {
         LOG.error("Error storing app: " + appId, e);
-        storedException = e;
-      } finally {
-        if (event.getType().equals(RMStateStoreEventType.STORE_APP)) {
-          notifyDoneStoringApplication(appId, storedException);
-        } else {
-          notifyDoneUpdatingApplication(appId, storedException);
-        }
+        notifyStoreOperationFailed(e);
       }
     } else if (event.getType().equals(RMStateStoreEventType.STORE_APP_ATTEMPT)
         || event.getType().equals(RMStateStoreEventType.UPDATE_APP_ATTEMPT)) {
@@ -589,24 +599,20 @@ public abstract class RMStateStore extends AbstractService {
         if (event.getType().equals(RMStateStoreEventType.STORE_APP_ATTEMPT)) {
           storeApplicationAttemptStateInternal(attemptState.getAttemptId()
             .toString(), attemptStateData);
+          notifyDoneStoringApplicationAttempt(attemptState.getAttemptId(),
+              storedException);
         } else {
           assert event.getType().equals(
             RMStateStoreEventType.UPDATE_APP_ATTEMPT);
           updateApplicationAttemptStateInternal(attemptState.getAttemptId()
             .toString(), attemptStateData);
-        }
-      } catch (Exception e) {
-        LOG
-          .error("Error storing appAttempt: " + attemptState.getAttemptId(), e);
-        storedException = e;
-      } finally {
-        if (event.getType().equals(RMStateStoreEventType.STORE_APP_ATTEMPT)) {
-          notifyDoneStoringApplicationAttempt(attemptState.getAttemptId(),
-            storedException);
-        } else {
           notifyDoneUpdatingApplicationAttempt(attemptState.getAttemptId(),
-            storedException);
+              storedException);
         }
+      } catch (Exception e) {
+        LOG.error(
+            "Error storing appAttempt: " + attemptState.getAttemptId(), e);
+        notifyStoreOperationFailed(e);
       }
     } else if (event.getType().equals(RMStateStoreEventType.REMOVE_APP)) {
       ApplicationState appState =
@@ -616,17 +622,34 @@ public abstract class RMStateStore extends AbstractService {
       LOG.info("Removing info for app: " + appId);
       try {
         removeApplicationState(appState);
+        notifyDoneRemovingApplcation(appId, removedException);
       } catch (Exception e) {
         LOG.error("Error removing app: " + appId, e);
-        removedException = e;
-      } finally {
-        notifyDoneRemovingApplcation(appId, removedException);
+        notifyStoreOperationFailed(e);
       }
     } else {
       LOG.error("Unknown RMStateStoreEvent type: " + event.getType());
     }
   }
 
+  @SuppressWarnings("unchecked")
+  /**
+   * In {#handleStoreEvent}, this method is called to notify the
+   * ResourceManager that the store operation has failed.
+   * @param failureCause the exception due to which the operation failed
+   */
+  private void notifyStoreOperationFailed(Exception failureCause) {
+    RMStateStoreOperationFailedEventType type;
+    if (failureCause instanceof StoreFencedException) {
+      type = RMStateStoreOperationFailedEventType.FENCED;
+    } else {
+      type = RMStateStoreOperationFailedEventType.FAILED;
+    }
+
+    rmDispatcher.getEventHandler().handle(
+        new RMStateStoreOperationFailedEvent(type, failureCause));
+  }
+
   @SuppressWarnings("unchecked")
   /**
    * In (@link handleStoreEvent}, this method is called to notify the

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

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

+ 23 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreOperationFailedEventType.java

@@ -0,0 +1,23 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.recovery;
+
+public enum RMStateStoreOperationFailedEventType {
+  FENCED, // Store operation failed because it was fenced
+  FAILED // Store operation failed for no known reason
+}

+ 28 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/StoreFencedException.java

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

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

@@ -23,7 +23,9 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.security.NoSuchAlgorithmException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
@@ -31,6 +33,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.io.DataInputByteBuffer;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
@@ -38,11 +41,14 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ZKUtil;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.client.RMHAServiceTarget;
+import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationAttemptStateDataProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationStateDataProto;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
 import org.apache.hadoop.yarn.util.ConverterUtils;
@@ -53,11 +59,14 @@ import org.apache.zookeeper.Op;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.Watcher.Event;
+import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.data.Stat;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
 
 @Private
 @Unstable
@@ -83,6 +92,55 @@ public class ZKRMStateStore extends RMStateStore {
   protected ZooKeeper zkClient;
   private ZooKeeper oldZkClient;
 
+  /** Fencing related variables */
+  private static final String FENCING_LOCK = "RM_ZK_FENCING_LOCK";
+  private String fencingNodePath;
+  private Op createFencingNodePathOp;
+  private Op deleteFencingNodePathOp;
+
+  @VisibleForTesting
+  List<ACL> zkRootNodeAcl;
+  private boolean useDefaultFencingScheme = false;
+  public static final int CREATE_DELETE_PERMS =
+      ZooDefs.Perms.CREATE | ZooDefs.Perms.DELETE;
+  private final String zkRootNodeAuthScheme =
+      new DigestAuthenticationProvider().getScheme();
+
+  private String zkRootNodeUsername;
+  private String zkRootNodePassword;
+
+  /**
+   * Given the {@link Configuration} and {@link ACL}s used (zkAcl) for
+   * ZooKeeper access, construct the {@link ACL}s for the store's root node.
+   * In the constructed {@link ACL}, all the users allowed by zkAcl are given
+   * rwa access, while the current RM has exclude create-delete access.
+   *
+   * To be called only when HA is enabled and the configuration doesn't set ACL
+   * for the root node.
+   */
+  @VisibleForTesting
+  @Private
+  @Unstable
+  protected List<ACL> constructZkRootNodeACL(
+      Configuration conf, List<ACL> sourceACLs) throws NoSuchAlgorithmException {
+    List<ACL> zkRootNodeAcl = new ArrayList<ACL>();
+    for (ACL acl : sourceACLs) {
+      zkRootNodeAcl.add(new ACL(
+          ZKUtil.removeSpecificPerms(acl.getPerms(), CREATE_DELETE_PERMS),
+          acl.getId()));
+    }
+
+    zkRootNodeUsername = HAUtil.getConfValueForRMInstance(
+        YarnConfiguration.RM_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_ADDRESS, conf);
+    zkRootNodePassword = Long.toString(ResourceManager.getClusterTimeStamp());
+    Id rmId = new Id(zkRootNodeAuthScheme,
+        DigestAuthenticationProvider.generateDigest(
+            zkRootNodeUsername + ":" + zkRootNodePassword));
+    zkRootNodeAcl.add(new ACL(CREATE_DELETE_PERMS, rmId));
+    return zkRootNodeAcl;
+  }
+
   @Override
   public synchronized void initInternal(Configuration conf) throws Exception {
     zkHostPort = conf.get(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS);
@@ -116,6 +174,29 @@ public class ZKRMStateStore extends RMStateStore {
     zkRootNodePath = znodeWorkingPath + "/" + ROOT_ZNODE_NAME;
     rmDTSecretManagerRoot = zkRootNodePath + "/" + RM_DT_SECRET_MANAGER_ROOT;
     rmAppRoot = zkRootNodePath + "/" + RM_APP_ROOT;
+
+    /* Initialize fencing related paths, acls, and ops */
+    fencingNodePath = zkRootNodePath + "/" + FENCING_LOCK;
+    createFencingNodePathOp = Op.create(fencingNodePath, new byte[0], zkAcl,
+        CreateMode.PERSISTENT);
+    deleteFencingNodePathOp = Op.delete(fencingNodePath, -1);
+    if (HAUtil.isHAEnabled(conf)) {
+      String zkRootNodeAclConf = HAUtil.getConfValueForRMInstance
+          (YarnConfiguration.ZK_RM_STATE_STORE_ROOT_NODE_ACL, conf);
+      if (zkRootNodeAclConf != null) {
+        zkRootNodeAclConf = ZKUtil.resolveConfIndirection(zkRootNodeAclConf);
+        try {
+          zkRootNodeAcl = ZKUtil.parseACLs(zkRootNodeAclConf);
+        } catch (ZKUtil.BadAclFormatException bafe) {
+          LOG.error("Invalid format for " +
+              YarnConfiguration.ZK_RM_STATE_STORE_ROOT_NODE_ACL);
+          throw bafe;
+        }
+      } else {
+        useDefaultFencingScheme = true;
+        zkRootNodeAcl = constructZkRootNodeACL(conf, zkAcl);
+      }
+    }
   }
 
   @Override
@@ -126,20 +207,76 @@ public class ZKRMStateStore extends RMStateStore {
     // ensure root dirs exist
     createRootDir(znodeWorkingPath);
     createRootDir(zkRootNodePath);
+    if (HAUtil.isHAEnabled(getConfig())){
+      fence();
+    }
     createRootDir(rmDTSecretManagerRoot);
     createRootDir(rmAppRoot);
   }
 
-  private void createRootDir(String rootPath) throws Exception {
+  private void createRootDir(final String rootPath) throws Exception {
+    // For root dirs, we shouldn't use the doMulti helper methods
     try {
-      createWithRetries(rootPath, null, zkAcl, CreateMode.PERSISTENT);
+      new ZKAction<String>() {
+        @Override
+        public String run() throws KeeperException, InterruptedException {
+          return zkClient.create(rootPath, null, zkAcl, CreateMode.PERSISTENT);
+        }
+      }.runWithRetries();
     } catch (KeeperException ke) {
-      if (ke.code() != Code.NODEEXISTS) {
+      if (ke.code() == Code.NODEEXISTS) {
+        LOG.debug(rootPath + "znode already exists!");
+      } else {
         throw ke;
       }
     }
   }
 
+  private void logRootNodeAcls(String prefix) throws KeeperException,
+      InterruptedException {
+    Stat getStat = new Stat();
+    List<ACL> getAcls = zkClient.getACL(zkRootNodePath, getStat);
+
+    StringBuilder builder = new StringBuilder();
+    builder.append(prefix);
+    for (ACL acl : getAcls) {
+      builder.append(acl.toString());
+    }
+    builder.append(getStat.toString());
+    LOG.debug(builder.toString());
+  }
+
+  private synchronized void fence() throws Exception {
+    if (LOG.isTraceEnabled()) {
+      logRootNodeAcls("Before fencing\n");
+    }
+
+    new ZKAction<Void>() {
+      @Override
+      public Void run() throws KeeperException, InterruptedException {
+        zkClient.setACL(zkRootNodePath, zkRootNodeAcl, -1);
+        return null;
+      }
+    }.runWithRetries();
+
+    // delete fencingnodepath
+    new ZKAction<Void>() {
+      @Override
+      public Void run() throws KeeperException, InterruptedException {
+        try {
+          zkClient.multi(Collections.singletonList(deleteFencingNodePathOp));
+        } catch (KeeperException.NoNodeException nne) {
+          LOG.info("Fencing node " + fencingNodePath + " doesn't exist to delete");
+        }
+        return null;
+      }
+    }.runWithRetries();
+
+    if (LOG.isTraceEnabled()) {
+      logRootNodeAcls("After fencing\n");
+    }
+  }
+
   private synchronized void closeZkClients() throws IOException {
     if (zkClient != null) {
       try {
@@ -176,7 +313,8 @@ public class ZKRMStateStore extends RMStateStore {
 
   private synchronized void loadRMDTSecretManagerState(RMState rmState)
       throws Exception {
-    List<String> childNodes = zkClient.getChildren(rmDTSecretManagerRoot, true);
+    List<String> childNodes =
+        getChildrenWithRetries(rmDTSecretManagerRoot, true);
 
     for (String childNodeName : childNodes) {
       if (childNodeName.startsWith(DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX)) {
@@ -209,7 +347,7 @@ public class ZKRMStateStore extends RMStateStore {
   }
 
   private synchronized void loadRMAppState(RMState rmState) throws Exception {
-    List<String> childNodes = zkClient.getChildren(rmAppRoot, true);
+    List<String> childNodes = getChildrenWithRetries(rmAppRoot, true);
     List<ApplicationAttemptState> attempts =
         new ArrayList<ApplicationAttemptState>();
     for (String childNodeName : childNodes) {
@@ -466,6 +604,8 @@ public class ZKRMStateStore extends RMStateStore {
   }
 
   @VisibleForTesting
+  @Private
+  @Unstable
   public synchronized void processWatchEvent(WatchedEvent event)
       throws Exception {
     Event.EventType eventType = event.getType();
@@ -506,65 +646,71 @@ public class ZKRMStateStore extends RMStateStore {
   }
 
   @VisibleForTesting
+  @Private
+  @Unstable
   String getNodePath(String root, String nodeName) {
     return (root + "/" + nodeName);
   }
 
-  @VisibleForTesting
-  public String createWithRetries(
-      final String path, final byte[] data, final List<ACL> acl,
-      final CreateMode mode) throws Exception {
-    return new ZKAction<String>() {
-      @Override
-      public String run() throws KeeperException, InterruptedException {
-        return zkClient.create(path, data, acl, mode);
-      }
-    }.runWithRetries();
-  }
-
-  private void deleteWithRetries(final String path, final int version)
-      throws Exception {
+  /**
+   * Helper method that creates fencing node, executes the passed operations,
+   * and deletes the fencing node.
+   */
+  private synchronized void doMultiWithRetries(
+      final List<Op> opList) throws Exception {
+    final List<Op> execOpList = new ArrayList<Op>(opList.size() + 2);
+    execOpList.add(createFencingNodePathOp);
+    execOpList.addAll(opList);
+    execOpList.add(deleteFencingNodePathOp);
     new ZKAction<Void>() {
       @Override
       public Void run() throws KeeperException, InterruptedException {
-        /**
-         * Call exists() to leave a watch on the node denoted by path.
-         * Delete node if exists. To pass the existence information to the
-         * caller, call delete irrespective of whether node exists or not.
-         */
-        if (zkClient.exists(path, true) == null) {
-          LOG.error("Trying to delete a path (" + path
-              + ") that doesn't exist.");
-        }
-        zkClient.delete(path, version);
+        zkClient.multi(execOpList);
         return null;
       }
     }.runWithRetries();
   }
 
-  private void doMultiWithRetries(final ArrayList<Op> opList) throws Exception {
-    new ZKAction<Void>() {
-      @Override
-      public Void run() throws KeeperException, InterruptedException {
-        zkClient.multi(opList);
-        return null;
+  /**
+   * Helper method that creates fencing node, executes the passed operation,
+   * and deletes the fencing node.
+   */
+  private void doMultiWithRetries(final Op op) throws Exception {
+    doMultiWithRetries(Collections.singletonList(op));
+  }
+
+  @VisibleForTesting
+  @Private
+  @Unstable
+  public void createWithRetries(
+      final String path, final byte[] data, final List<ACL> acl,
+      final CreateMode mode) throws Exception {
+    doMultiWithRetries(Op.create(path, data, acl, mode));
+  }
+
+  private void deleteWithRetries(final String path, final int version)
+      throws Exception {
+    try {
+      doMultiWithRetries(Op.delete(path, version));
+    } catch (KeeperException.NoNodeException nne) {
+      // We tried to delete a node that doesn't exist
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Attempted to delete a non-existing znode " + path);
       }
-    }.runWithRetries();
+    }
   }
 
   @VisibleForTesting
+  @Private
+  @Unstable
   public void setDataWithRetries(final String path, final byte[] data,
                                  final int version) throws Exception {
-    new ZKAction<Void>() {
-      @Override
-      public Void run() throws KeeperException, InterruptedException {
-        zkClient.setData(path, data, version);
-        return null;
-      }
-    }.runWithRetries();
+    doMultiWithRetries(Op.setData(path, data, version));
   }
 
   @VisibleForTesting
+  @Private
+  @Unstable
   public byte[] getDataWithRetries(final String path, final boolean watch)
       throws Exception {
     return new ZKAction<byte[]>() {
@@ -576,6 +722,16 @@ public class ZKRMStateStore extends RMStateStore {
     }.runWithRetries();
   }
 
+  private List<String> getChildrenWithRetries(
+      final String path, final boolean watch) throws Exception {
+    return new ZKAction<List<String>>() {
+      @Override
+      List<String> run() throws KeeperException, InterruptedException {
+        return zkClient.getChildren(path, watch);
+      }
+    }.runWithRetries();
+  }
+
   private abstract class ZKAction<T> {
     // run() expects synchronization on ZKRMStateStore.this
     abstract T run() throws KeeperException, InterruptedException;
@@ -596,11 +752,29 @@ public class ZKRMStateStore extends RMStateStore {
       }
     }
 
+    private boolean shouldRetry(Code code) {
+      switch (code) {
+        case CONNECTIONLOSS:
+        case OPERATIONTIMEOUT:
+          return true;
+        default:
+          break;
+      }
+      return false;
+    }
+
     T runWithRetries() throws Exception {
       int retry = 0;
       while (true) {
         try {
           return runWithCheck();
+        } catch (KeeperException.NoAuthException nae) {
+          if (HAUtil.isHAEnabled(getConfig())) {
+            // NoAuthException possibly means that this store is fenced due to
+            // another RM becoming active. Even if not,
+            // it is safer to assume we have been fenced
+            throw new StoreFencedException();
+          }
         } catch (KeeperException ke) {
           if (shouldRetry(ke.code()) && ++retry < numRetries) {
             continue;
@@ -611,17 +785,6 @@ public class ZKRMStateStore extends RMStateStore {
     }
   }
 
-  private static boolean shouldRetry(Code code) {
-    switch (code) {
-      case CONNECTIONLOSS:
-      case OPERATIONTIMEOUT:
-        return true;
-      default:
-        break;
-    }
-    return false;
-  }
-
   private synchronized void createConnection()
       throws IOException, InterruptedException {
     closeZkClients();
@@ -629,6 +792,10 @@ public class ZKRMStateStore extends RMStateStore {
         retries++) {
       try {
         zkClient = getNewZooKeeper();
+        if (useDefaultFencingScheme) {
+          zkClient.addAuthInfo(zkRootNodeAuthScheme,
+              (zkRootNodeUsername + ":" + zkRootNodePassword).getBytes());
+        }
       } catch (IOException ioe) {
         // Retry in case of network failures
         LOG.info("Failed to connect to the ZooKeeper on attempt - " +
@@ -646,6 +813,8 @@ public class ZKRMStateStore extends RMStateStore {
 
   // protected to mock for testing
   @VisibleForTesting
+  @Private
+  @Unstable
   protected synchronized ZooKeeper getNewZooKeeper()
       throws IOException, InterruptedException {
     ZooKeeper zk = new ZooKeeper(zkHostPort, zkSessionTimeout, null);

+ 16 - 15
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

@@ -136,9 +136,6 @@ public class FairScheduler implements ResourceScheduler {
   // How often fair shares are re-calculated (ms)
   protected long UPDATE_INTERVAL = 500;
 
-  // Whether to use username in place of "default" queue name
-  private volatile boolean userAsDefaultQueue = false;
-
   private final static List<Container> EMPTY_CONTAINER_LIST =
       new ArrayList<Container>();
 
@@ -640,6 +637,12 @@ public class FairScheduler implements ResourceScheduler {
     RMApp rmApp = rmContext.getRMApps().get(
         applicationAttemptId.getApplicationId());
     FSLeafQueue queue = assignToQueue(rmApp, queueName, user);
+    if (queue == null) {
+      rmContext.getDispatcher().getEventHandler().handle(
+          new RMAppAttemptRejectedEvent(applicationAttemptId,
+              "Application rejected by queue placement policy"));
+      return;
+    }
 
     FSSchedulerApp schedulerApp =
         new FSSchedulerApp(applicationAttemptId, user,
@@ -675,17 +678,16 @@ public class FairScheduler implements ResourceScheduler {
   
   @VisibleForTesting
   FSLeafQueue assignToQueue(RMApp rmApp, String queueName, String user) {
-    // Potentially set queue to username if configured to do so
-    if (queueName.equals(YarnConfiguration.DEFAULT_QUEUE_NAME) &&
-        userAsDefaultQueue) {
-      queueName = user;
-    }
-    
-    FSLeafQueue queue = queueMgr.getLeafQueue(queueName,
-        conf.getAllowUndeclaredPools());
-    if (queue == null) {
-      // queue is not an existing or createable leaf queue
-      queue = queueMgr.getLeafQueue(YarnConfiguration.DEFAULT_QUEUE_NAME, false);
+    FSLeafQueue queue = null;
+    try {
+      QueuePlacementPolicy policy = queueMgr.getPlacementPolicy();
+      queueName = policy.assignAppToQueue(queueName, user);
+      if (queueName == null) {
+        return null;
+      }
+      queue = queueMgr.getLeafQueue(queueName, true);
+    } catch (IOException ex) {
+      LOG.error("Error assigning app to queue, rejecting", ex);
     }
     
     if (rmApp != null) {
@@ -1155,7 +1157,6 @@ public class FairScheduler implements ResourceScheduler {
     minimumAllocation = this.conf.getMinimumAllocation();
     maximumAllocation = this.conf.getMaximumAllocation();
     incrAllocation = this.conf.getIncrementAllocation();
-    userAsDefaultQueue = this.conf.getUserAsDefaultQueue();
     continuousSchedulingEnabled = this.conf.isContinuousSchedulingEnabled();
     continuousSchedulingSleepMs =
             this.conf.getContinuousSchedulingSleepMs();

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

@@ -25,6 +25,7 @@ import java.net.URLConnection;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.CopyOnWriteArrayList;
@@ -51,6 +52,8 @@ import org.w3c.dom.NodeList;
 import org.w3c.dom.Text;
 import org.xml.sax.SAXException;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Maintains a list of queues as well as scheduling parameters for each queue,
  * such as guaranteed share allocations, from the fair scheduler config file.
@@ -87,6 +90,8 @@ public class QueueManager {
   private FSParentQueue rootQueue;
 
   private volatile QueueManagerInfo info = new QueueManagerInfo();
+  @VisibleForTesting
+  volatile QueuePlacementPolicy placementPolicy;
   
   private long lastReloadAttempt; // Last time we tried to reload the queues file
   private long lastSuccessfulReload; // Last time we successfully reloaded queues
@@ -107,6 +112,8 @@ public class QueueManager {
     queues.put(rootQueue.getName(), rootQueue);
     
     this.allocFile = conf.getAllocationFile();
+    placementPolicy = new QueuePlacementPolicy(getSimplePlacementRules(),
+        new HashSet<String>(), conf);
     
     reloadAllocs();
     lastSuccessfulReload = scheduler.getClock().getTime();
@@ -115,6 +122,28 @@ public class QueueManager {
     getLeafQueue(YarnConfiguration.DEFAULT_QUEUE_NAME, true);
   }
   
+  public void updatePlacementPolicy(FairSchedulerConfiguration conf) {
+    
+  }
+  
+  /**
+   * Construct simple queue placement policy from allow-undeclared-pools and
+   * user-as-default-queue.
+   */
+  private List<QueuePlacementRule> getSimplePlacementRules() {
+    boolean create = scheduler.getConf().getAllowUndeclaredPools();
+    boolean userAsDefaultQueue = scheduler.getConf().getUserAsDefaultQueue();
+    List<QueuePlacementRule> rules = new ArrayList<QueuePlacementRule>();
+    rules.add(new QueuePlacementRule.Specified().initialize(create, null));
+    if (userAsDefaultQueue) {
+      rules.add(new QueuePlacementRule.User().initialize(create, null));
+    }
+    if (!userAsDefaultQueue || !create) {
+      rules.add(new QueuePlacementRule.Default().initialize(true, null));
+    }
+    return rules;
+  }
+  
   /**
    * Get a queue by name, creating it if the create param is true and is necessary.
    * If the queue is not or can not be a leaf queue, i.e. it already exists as a
@@ -226,6 +255,10 @@ public class QueueManager {
       return queues.containsKey(name);
     }
   }
+  
+  public QueuePlacementPolicy getPlacementPolicy() {
+    return placementPolicy;
+  }
 
   /**
    * Reload allocations file if it hasn't been loaded in a while
@@ -290,6 +323,8 @@ public class QueueManager {
     long fairSharePreemptionTimeout = Long.MAX_VALUE;
     long defaultMinSharePreemptionTimeout = Long.MAX_VALUE;
     SchedulingPolicy defaultSchedPolicy = SchedulingPolicy.getDefault();
+    
+    QueuePlacementPolicy newPlacementPolicy = null;
 
     // Remember all queue names so we can display them on web UI, etc.
     List<String> queueNamesInAllocFile = new ArrayList<String>();
@@ -306,6 +341,7 @@ public class QueueManager {
           "file: top-level element not <allocations>");
     NodeList elements = root.getChildNodes();
     List<Element> queueElements = new ArrayList<Element>();
+    Element placementPolicyElement = null;
     for (int i = 0; i < elements.getLength(); i++) {
       Node node = elements.item(i);
       if (node instanceof Element) {
@@ -348,6 +384,8 @@ public class QueueManager {
           String text = ((Text)element.getFirstChild()).getData().trim();
           SchedulingPolicy.setDefault(text);
           defaultSchedPolicy = SchedulingPolicy.getDefault();
+        } else if ("queuePlacementPolicy".equals(element.getTagName())) {
+          placementPolicyElement = element;
         } else {
           LOG.warn("Bad element in allocations file: " + element.getTagName());
         }
@@ -369,6 +407,15 @@ public class QueueManager {
           userMaxApps, queueWeights, queuePolicies, minSharePreemptionTimeouts,
           queueAcls, queueNamesInAllocFile);
     }
+    
+    // Load placement policy and pass it configured queues
+    if (placementPolicyElement != null) {
+      newPlacementPolicy = QueuePlacementPolicy.fromXml(placementPolicyElement,
+          new HashSet<String>(queueNamesInAllocFile), scheduler.getConf());
+    } else {
+      newPlacementPolicy = new QueuePlacementPolicy(getSimplePlacementRules(),
+          new HashSet<String>(queueNamesInAllocFile), scheduler.getConf());
+    }
 
     // Commit the reload; also create any queue defined in the alloc file
     // if it does not already exist, so it can be displayed on the web UI.
@@ -377,6 +424,7 @@ public class QueueManager {
           queueMaxApps, userMaxApps, queueWeights, userMaxAppsDefault,
           queueMaxAppsDefault, defaultSchedPolicy, minSharePreemptionTimeouts,
           queueAcls, fairSharePreemptionTimeout, defaultMinSharePreemptionTimeout);
+      placementPolicy = newPlacementPolicy;
       
       // Make sure all queues exist
       for (String name: queueNamesInAllocFile) {

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

@@ -0,0 +1,121 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.Groups;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+public class QueuePlacementPolicy {
+  private static final Map<String, Class<? extends QueuePlacementRule>> ruleClasses;
+  static {
+    Map<String, Class<? extends QueuePlacementRule>> map =
+        new HashMap<String, Class<? extends QueuePlacementRule>>();
+    map.put("user", QueuePlacementRule.User.class);
+    map.put("primaryGroup", QueuePlacementRule.PrimaryGroup.class);
+    map.put("specified", QueuePlacementRule.Specified.class);
+    map.put("default", QueuePlacementRule.Default.class);
+    map.put("reject", QueuePlacementRule.Reject.class);
+    ruleClasses = Collections.unmodifiableMap(map);
+  }
+  
+  private final List<QueuePlacementRule> rules;
+  private final Set<String> configuredQueues;
+  private final Groups groups;
+  
+  public QueuePlacementPolicy(List<QueuePlacementRule> rules,
+      Set<String> configuredQueues, Configuration conf)
+      throws AllocationConfigurationException {
+    for (int i = 0; i < rules.size()-1; i++) {
+      if (rules.get(i).isTerminal()) {
+        throw new AllocationConfigurationException("Rules after rule "
+            + i + " in queue placement policy can never be reached");
+      }
+    }
+    if (!rules.get(rules.size()-1).isTerminal()) {
+      throw new AllocationConfigurationException(
+          "Could get past last queue placement rule without assigning");
+    }
+    this.rules = rules;
+    this.configuredQueues = configuredQueues;
+    groups = new Groups(conf);
+  }
+  
+  /**
+   * Builds a QueuePlacementPolicy from an xml element.
+   */
+  public static QueuePlacementPolicy fromXml(Element el, Set<String> configuredQueues,
+      Configuration conf) throws AllocationConfigurationException {
+    List<QueuePlacementRule> rules = new ArrayList<QueuePlacementRule>();
+    NodeList elements = el.getChildNodes();
+    for (int i = 0; i < elements.getLength(); i++) {
+      Node node = elements.item(i);
+      if (node instanceof Element) {
+        Element element = (Element)node;
+        String ruleName = element.getTagName();
+        Class<? extends QueuePlacementRule> clazz = ruleClasses.get(ruleName);
+        if (clazz == null) {
+          throw new AllocationConfigurationException("No rule class found for "
+              + ruleName);
+        }
+        QueuePlacementRule rule = ReflectionUtils.newInstance(clazz, null);
+        rule.initializeFromXml(element);
+        rules.add(rule);
+      }
+    }
+    return new QueuePlacementPolicy(rules, configuredQueues, conf);
+  }
+  
+  /**
+   * Applies this rule to an app with the given requested queue and user/group
+   * information.
+   * 
+   * @param requestedQueue
+   *    The queue specified in the ApplicationSubmissionContext
+   * @param user
+   *    The user submitting the app
+   * @return
+   *    The name of the queue to assign the app to.  Or null if the app should
+   *    be rejected.
+   * @throws IOException
+   *    If an exception is encountered while getting the user's groups
+   */
+  public String assignAppToQueue(String requestedQueue, String user)
+      throws IOException {
+    for (QueuePlacementRule rule : rules) {
+      String queue = rule.assignAppToQueue(requestedQueue, user, groups,
+          configuredQueues);
+      if (queue == null || !queue.isEmpty()) {
+        return queue;
+      }
+    }
+    throw new IllegalStateException("Should have applied a rule before " +
+    		"reaching here");
+  }
+}

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

@@ -0,0 +1,200 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.security.Groups;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.w3c.dom.Element;
+import org.w3c.dom.NamedNodeMap;
+import org.w3c.dom.Node;
+
+public abstract class QueuePlacementRule {
+  protected boolean create;
+  
+  /**
+   * Initializes the rule with any arguments.
+   * 
+   * @param args
+   *    Additional attributes of the rule's xml element other than create.
+   */
+  public QueuePlacementRule initialize(boolean create, Map<String, String> args) {
+    this.create = create;
+    return this;
+  }
+  
+  /**
+   * 
+   * @param requestedQueue
+   *    The queue explicitly requested.
+   * @param user
+   *    The user submitting the app.
+   * @param groups
+   *    The groups of the user submitting the app.
+   * @param configuredQueues
+   *    The queues specified in the scheduler configuration.
+   * @return
+   *    The queue to place the app into. An empty string indicates that we should
+   *    continue to the next rule, and null indicates that the app should be rejected.
+   */
+  public String assignAppToQueue(String requestedQueue, String user,
+      Groups groups, Collection<String> configuredQueues) throws IOException {
+    String queue = getQueueForApp(requestedQueue, user, groups);
+    if (create || configuredQueues.contains(queue)) {
+      return queue;
+    } else {
+      return "";
+    }
+  }
+  
+  public void initializeFromXml(Element el) {
+    boolean create = true;
+    NamedNodeMap attributes = el.getAttributes();
+    Map<String, String> args = new HashMap<String, String>();
+    for (int i = 0; i < attributes.getLength(); i++) {
+      Node node = attributes.item(i);
+      String key = node.getNodeName();
+      String value = node.getNodeValue();
+      if (key.equals("create")) {
+        create = Boolean.parseBoolean(value);
+      } else {
+        args.put(key, value);
+      }
+    }
+    initialize(create, args);
+  }
+  
+  /**
+   * Returns true if this rule never tells the policy to continue.
+   */
+  public abstract boolean isTerminal();
+  
+  /**
+   * Applies this rule to an app with the given requested queue and user/group
+   * information.
+   * 
+   * @param requestedQueue
+   *    The queue specified in the ApplicationSubmissionContext
+   * @param user
+   *    The user submitting the app.
+   * @param groups
+   *    The groups of the user submitting the app.
+   * @return
+   *    The name of the queue to assign the app to, or null to empty string
+   *    continue to the next rule.
+   */
+  protected abstract String getQueueForApp(String requestedQueue, String user,
+      Groups groups) throws IOException;
+
+  /**
+   * Places apps in queues by username of the submitter
+   */
+  public static class User extends QueuePlacementRule {
+    @Override
+    protected String getQueueForApp(String requestedQueue,
+        String user, Groups groups) {
+      return "root." + user;
+    }
+    
+    @Override
+    public boolean isTerminal() {
+      return create;
+    }
+  }
+  
+  /**
+   * Places apps in queues by primary group of the submitter
+   */
+  public static class PrimaryGroup extends QueuePlacementRule {
+    @Override
+    protected String getQueueForApp(String requestedQueue,
+        String user, Groups groups) throws IOException {
+      return "root." + groups.getGroups(user).get(0);
+    }
+    
+    @Override
+    public boolean isTerminal() {
+      return create;
+    }
+  }
+
+  /**
+   * Places apps in queues by requested queue of the submitter
+   */
+  public static class Specified extends QueuePlacementRule {
+    @Override
+    protected String getQueueForApp(String requestedQueue,
+        String user, Groups groups) {
+      if (requestedQueue.equals(YarnConfiguration.DEFAULT_QUEUE_NAME)) {
+        return "";
+      } else {
+        if (!requestedQueue.startsWith("root.")) {
+          requestedQueue = "root." + requestedQueue;
+        }
+        return requestedQueue;
+      }
+    }
+    
+    @Override
+    public boolean isTerminal() {
+      return false;
+    }
+  }
+  
+  /**
+   * Places all apps in the default queue
+   */
+  public static class Default extends QueuePlacementRule {
+    @Override
+    protected String getQueueForApp(String requestedQueue, String user,
+        Groups groups) {
+      return "root." + YarnConfiguration.DEFAULT_QUEUE_NAME;
+    }
+    
+    @Override
+    public boolean isTerminal() {
+      return create;
+    }
+  }
+  
+  /**
+   * Rejects all apps
+   */
+  public static class Reject extends QueuePlacementRule {
+    @Override
+    public String assignAppToQueue(String requestedQueue, String user,
+        Groups groups, Collection<String> configuredQueues) {
+      return null;
+    }
+    
+    @Override
+    protected String getQueueForApp(String requestedQueue, String user,
+        Groups groups) {
+      throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public boolean isTerminal() {
+      return true;
+    }
+  }
+}

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

@@ -18,15 +18,32 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.recovery;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HAServiceProtocol;
+import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.zookeeper.ZooKeeper;
 import org.junit.Test;
 
@@ -56,7 +73,7 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
 
     public RMStateStore getRMStateStore() throws Exception {
       String workingZnode = "/Test";
-      YarnConfiguration conf = new YarnConfiguration();
+      Configuration conf = new YarnConfiguration();
       conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS, hostPort);
       conf.set(YarnConfiguration.ZK_RM_STATE_STORE_PARENT_PATH, workingZnode);
       this.client = createClient();
@@ -77,4 +94,81 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
     testRMAppStateStore(zkTester);
     testRMDTSecretManagerStateStore(zkTester);
   }
+
+  private Configuration createHARMConf(
+      String rmIds, String rmId, int adminPort) {
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
+    conf.set(YarnConfiguration.RM_HA_IDS, rmIds);
+    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    conf.set(YarnConfiguration.RM_STORE, ZKRMStateStore.class.getName());
+    conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS, hostPort);
+    conf.set(YarnConfiguration.RM_HA_ID, rmId);
+    for (String rpcAddress : HAUtil.RPC_ADDRESS_CONF_KEYS) {
+      conf.set(HAUtil.addSuffix(rpcAddress, rmId), "localhost:0");
+    }
+    conf.set(YarnConfiguration.RM_HA_ADMIN_ADDRESS, "localhost:" + adminPort);
+    return conf;
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testFencing() throws Exception {
+    StateChangeRequestInfo req = new StateChangeRequestInfo(
+        HAServiceProtocol.RequestSource.REQUEST_BY_USER);
+
+    Configuration conf1 = createHARMConf("rm1,rm2", "rm1", 1234);
+    ResourceManager rm1 = new ResourceManager();
+    rm1.init(conf1);
+    rm1.start();
+    rm1.getHAService().transitionToActive(req);
+    assertEquals("RM with ZKStore didn't start",
+        Service.STATE.STARTED, rm1.getServiceState());
+    assertEquals("RM should be Active",
+        HAServiceProtocol.HAServiceState.ACTIVE,
+        rm1.getHAService().getServiceStatus().getState());
+
+    Configuration conf2 = createHARMConf("rm1,rm2", "rm2", 5678);
+    ResourceManager rm2 = new ResourceManager();
+    rm2.init(conf2);
+    rm2.start();
+    rm2.getHAService().transitionToActive(req);
+    assertEquals("RM with ZKStore didn't start",
+        Service.STATE.STARTED, rm2.getServiceState());
+    assertEquals("RM should be Active",
+        HAServiceProtocol.HAServiceState.ACTIVE,
+        rm2.getHAService().getServiceStatus().getState());
+
+    // Submitting an application to RM1 to trigger a state store operation.
+    // RM1 should realize that it got fenced and is not the Active RM anymore.
+    Map mockMap = mock(Map.class);
+    ApplicationSubmissionContext asc =
+        ApplicationSubmissionContext.newInstance(
+            ApplicationId.newInstance(1000, 1),
+            "testApplication", // app Name
+            "default", // queue name
+            Priority.newInstance(0),
+            ContainerLaunchContext.newInstance(mockMap, mockMap,
+                new ArrayList<String>(), mockMap, mock(ByteBuffer.class),
+                mockMap),
+            false, // unmanaged AM
+            true, // cancelTokens
+            1, // max app attempts
+            Resource.newInstance(1024, 1));
+    ClientRMService rmService = rm1.getClientRMService();
+    rmService.submitApplication(SubmitApplicationRequest.newInstance(asc));
+
+    for (int i = 0; i < 30; i++) {
+      if (HAServiceProtocol.HAServiceState.ACTIVE == rm1.getHAService()
+          .getServiceStatus().getState()) {
+        Thread.sleep(100);
+      }
+    }
+    assertEquals("RM should have been fenced",
+        HAServiceProtocol.HAServiceState.STANDBY,
+        rm1.getHAService().getServiceStatus().getState());
+    assertEquals("RM should be Active",
+        HAServiceProtocol.HAServiceState.ACTIVE,
+        rm2.getHAService().getServiceStatus().getState());
+  }
 }

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

@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.security.GroupMappingServiceProvider;
+
+public class SimpleGroupsMapping implements GroupMappingServiceProvider {
+  
+  @Override
+  public List<String> getGroups(String user) {
+    return Arrays.asList(user + "group");
+  }
+
+  @Override
+  public void cacheGroupsRefresh() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void cacheGroupsAdd(List<String> groups) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+}

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

@@ -44,7 +44,9 @@ import javax.xml.parsers.ParserConfigurationException;
 import junit.framework.Assert;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.security.GroupMappingServiceProvider;
 import org.apache.hadoop.yarn.MockApps;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -94,6 +96,8 @@ import org.junit.Before;
 import org.junit.Test;
 import org.xml.sax.SAXException;
 
+import com.google.common.collect.Sets;
+
 public class TestFairScheduler {
 
   private class MockClock implements Clock {
@@ -616,6 +620,7 @@ public class TestFairScheduler {
 
     conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "false");
     scheduler.reinitialize(conf, resourceManager.getRMContext());
+    scheduler.getQueueManager().initialize();
     AppAddedSchedulerEvent appAddedEvent2 = new AppAddedSchedulerEvent(
         createAppAttemptId(2, 1), "default", "user2");
     scheduler.handle(appAddedEvent2);
@@ -664,6 +669,46 @@ public class TestFairScheduler {
     assertEquals(rmApp2.getQueue(), queue2.getName());
     assertEquals("root.notdefault", rmApp2.getQueue());
   }
+  
+  @Test
+  public void testQueuePlacementWithPolicy() throws Exception {
+    Configuration conf = createConfiguration();
+    conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
+        SimpleGroupsMapping.class, GroupMappingServiceProvider.class);
+    ApplicationAttemptId appId;
+    Map<ApplicationAttemptId, FSSchedulerApp> apps = scheduler.applications;
+
+    List<QueuePlacementRule> rules = new ArrayList<QueuePlacementRule>();
+    rules.add(new QueuePlacementRule.Specified().initialize(true, null));
+    rules.add(new QueuePlacementRule.User().initialize(false, null));
+    rules.add(new QueuePlacementRule.PrimaryGroup().initialize(false, null));
+    rules.add(new QueuePlacementRule.Default().initialize(true, null));
+    Set<String> queues = Sets.newHashSet("root.user1", "root.user3group");
+    scheduler.getQueueManager().placementPolicy = new QueuePlacementPolicy(
+        rules, queues, conf);
+    appId = createSchedulingRequest(1024, "somequeue", "user1");
+    assertEquals("root.somequeue", apps.get(appId).getQueueName());
+    appId = createSchedulingRequest(1024, "default", "user1");
+    assertEquals("root.user1", apps.get(appId).getQueueName());
+    appId = createSchedulingRequest(1024, "default", "user3");
+    assertEquals("root.user3group", apps.get(appId).getQueueName());
+    appId = createSchedulingRequest(1024, "default", "otheruser");
+    assertEquals("root.default", apps.get(appId).getQueueName());
+    
+    // test without specified as first rule
+    rules = new ArrayList<QueuePlacementRule>();
+    rules.add(new QueuePlacementRule.User().initialize(false, null));
+    rules.add(new QueuePlacementRule.Specified().initialize(true, null));
+    rules.add(new QueuePlacementRule.Default().initialize(true, null));
+    scheduler.getQueueManager().placementPolicy = new QueuePlacementPolicy(
+        rules, queues, conf);
+    appId = createSchedulingRequest(1024, "somequeue", "user1");
+    assertEquals("root.user1", apps.get(appId).getQueueName());
+    appId = createSchedulingRequest(1024, "somequeue", "otheruser");
+    assertEquals("root.somequeue", apps.get(appId).getQueueName());
+    appId = createSchedulingRequest(1024, "default", "otheruser");
+    assertEquals("root.default", apps.get(appId).getQueueName());
+  }
 
   @Test
   public void testFairShareWithMinAlloc() throws Exception {

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

@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Set;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.security.GroupMappingServiceProvider;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+
+import com.google.common.collect.Sets;
+
+public class TestQueuePlacementPolicy {
+  private final static Configuration conf = new Configuration();
+  private final static Set<String> configuredQueues = Sets.newHashSet("root.someuser");
+  
+  @BeforeClass
+  public static void setup() {
+    conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
+        SimpleGroupsMapping.class, GroupMappingServiceProvider.class);
+  }
+  
+  @Test
+  public void testSpecifiedUserPolicy() throws Exception {
+    StringBuffer sb = new StringBuffer();
+    sb.append("<queuePlacementPolicy>");
+    sb.append("  <specified />");
+    sb.append("  <user />");
+    sb.append("</queuePlacementPolicy>");
+    QueuePlacementPolicy policy = parse(sb.toString());
+    assertEquals("root.specifiedq",policy.assignAppToQueue("specifiedq", "someuser"));
+    assertEquals("root.someuser", policy.assignAppToQueue("default", "someuser"));
+    assertEquals("root.otheruser", policy.assignAppToQueue("default", "otheruser"));
+  }
+  
+  @Test
+  public void testNoCreate() throws Exception {
+    StringBuffer sb = new StringBuffer();
+    sb.append("<queuePlacementPolicy>");
+    sb.append("  <specified />");
+    sb.append("  <user create=\"false\" />");
+    sb.append("  <default />");
+    sb.append("</queuePlacementPolicy>");
+    QueuePlacementPolicy policy = parse(sb.toString());
+    assertEquals("root.specifiedq", policy.assignAppToQueue("specifiedq", "someuser"));
+    assertEquals("root.someuser", policy.assignAppToQueue("default", "someuser"));
+    assertEquals("root.specifiedq", policy.assignAppToQueue("specifiedq", "otheruser"));
+    assertEquals("root.default", policy.assignAppToQueue("default", "otheruser"));
+  }
+  
+  @Test
+  public void testSpecifiedThenReject() throws Exception {
+    StringBuffer sb = new StringBuffer();
+    sb.append("<queuePlacementPolicy>");
+    sb.append("  <specified />");
+    sb.append("  <reject />");
+    sb.append("</queuePlacementPolicy>");
+    QueuePlacementPolicy policy = parse(sb.toString());
+    assertEquals("root.specifiedq", policy.assignAppToQueue("specifiedq", "someuser"));
+    assertEquals(null, policy.assignAppToQueue("default", "someuser"));
+  }
+  
+  @Test (expected = AllocationConfigurationException.class)
+  public void testOmittedTerminalRule() throws Exception {
+    StringBuffer sb = new StringBuffer();
+    sb.append("<queuePlacementPolicy>");
+    sb.append("  <specified />");
+    sb.append("  <user create=\"false\" />");
+    sb.append("</queuePlacementPolicy>");
+    parse(sb.toString());
+  }
+  
+  @Test (expected = AllocationConfigurationException.class)
+  public void testTerminalRuleInMiddle() throws Exception {
+    StringBuffer sb = new StringBuffer();
+    sb.append("<queuePlacementPolicy>");
+    sb.append("  <specified />");
+    sb.append("  <default />");
+    sb.append("  <user />");
+    sb.append("</queuePlacementPolicy>");
+    parse(sb.toString());
+  }
+  
+  private QueuePlacementPolicy parse(String str) throws Exception {
+    // Read and parse the allocations file.
+    DocumentBuilderFactory docBuilderFactory =
+      DocumentBuilderFactory.newInstance();
+    docBuilderFactory.setIgnoringComments(true);
+    DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
+    Document doc = builder.parse(IOUtils.toInputStream(str));
+    Element root = doc.getDocumentElement();
+    return QueuePlacementPolicy.fromXml(root, configuredQueues, conf);
+  }
+}

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

@@ -101,6 +101,16 @@ Hadoop MapReduce Next Generation - Fair Scheduler
   Fair Scheduler. Among them, is the use of a custom policies governing 
   priority “boosting” over  certain apps. 
 
+* {Automatically placing applications in queues}
+
+  The Fair Scheduler allows administrators to configure policies that
+  automatically place submitted applications into appropriate queues. Placement
+  can depend on the user and groups of the submitter and the requested queue
+  passed by the application. A policy consists of a set of rules that are applied
+  sequentially to classify an incoming application. Each rule either places the
+  app into a queue, rejects it, or continues on to the next rule. Refer to the
+  allocation file format below for how to configure these policies.
+
 * {Installation}
 
   To use the Fair Scheduler first assign the appropriate scheduler class in 
@@ -138,7 +148,8 @@ Properties that can be placed in yarn-site.xml
     * Whether to use the username associated with the allocation as the default 
       queue name, in the event that a queue name is not specified. If this is set 
       to "false" or unset, all jobs have a shared default queue, named "default".
-      Defaults to true.
+      Defaults to true.  If a queue placement policy is given in the allocations
+      file, this property is ignored.
 
  * <<<yarn.scheduler.fair.preemption>>>
 
@@ -180,6 +191,16 @@ Properties that can be placed in yarn-site.xml
       opportunities to pass up. The default value of -1.0 means don't pass up any
       scheduling opportunities.
 
+ * <<<yarn.scheduler.fair.allow-undeclared-pools>>>
+
+    * If this is true, new queues can be created at application submission time,
+      whether because they are specified as the application's queue by the
+      submitter or because they are placed there by the user-as-default-queue
+      property. If this is false, any time an app would be placed in a queue that
+      is not specified in the allocations file, it is placed in the "default" queue
+      instead. Defaults to true. If a queue placement policy is given in the
+      allocations file, this property is ignored.
+
 Allocation file format
 
   The allocation file must be in XML format. The format contains five types of
@@ -248,25 +269,29 @@ Allocation file format
    policy for queues; overriden by the schedulingPolicy element in each queue
    if specified. Defaults to "fair".
 
-  An example allocation file is given here:
+ * <<A queuePlacementPolicy element>>, which contains a list of rule elements
+   that tell the scheduler how to place incoming apps into queues. Rules
+   are applied in the order that they are listed. Rules may take arguments. All
+   rules accept the "create" argument, which indicates whether the rule can create
+   a new queue. "Create" defaults to true; if set to false and the rule would
+   place the app in a queue that is not configured in the allocations file, we
+   continue on to the next rule. The last rule must be one that can never issue a
+   continue.  Valid rules are:
 
-Queue Access Control Lists (ACLs)
+     * specified: the app is placed into the queue it requested.  If the app
+       requested no queue, i.e. it specified "default", we continue.
 
-  Queue Access Control Lists (ACLs) allow administrators to control who may
-  take actions on particular queues. They are configured with the aclSubmitApps
-  and aclAdministerApps properties, which can be set per queue. Currently the
-  only supported administrative action is killing an application. Anybody who
-  may administer a queue may also submit applications to it. These properties
-  take values in a format like "user1,user2 group1,group2" or " group1,group2".
-  An action on a queue will be permitted if its user or group is in the ACL of
-  that queue or in the ACL of any of that queue's ancestors. So if queue2
-  is inside queue1, and user1 is in queue1's ACL, and user2 is in queue2's
-  ACL, then both users may submit to queue2.
-  
-  The root queue's ACLs are "*" by default which, because ACLs are passed down,
-  means that everybody may submit to and kill applications from every queue.
-  To start restricting access, change the root queue's ACLs to something other
-  than "*". 
+     * user: the app is placed into a queue with the name of the user who
+       submitted it.
+
+     * primaryGroup: the app is placed into a queue with the name of the
+       primary group of the user who submitted it.
+
+     * default: the app is placed into the queue named "default".
+
+     * reject: the app is rejected.
+
+  An example allocation file is given here:
 
 ---
 <?xml version="1.0"?>
@@ -282,14 +307,41 @@ Queue Access Control Lists (ACLs)
       <minResources>5000 mb,0vcores</minResources>
     </queue>
   </queue>
+  
   <user name="sample_user">
     <maxRunningApps>30</maxRunningApps>
   </user>
   <userMaxAppsDefault>5</userMaxAppsDefault>
+  
+  <queuePlacementPolicy>
+    <specified />
+    <primarygroup create="false" />
+    <default />
+  </queuePlacementPolicy>
 </allocations>
 ---
 
   Note that for backwards compatibility with the original FairScheduler, "queue" elements can instead be named as "pool" elements.
+
+
+Queue Access Control Lists (ACLs)
+
+  Queue Access Control Lists (ACLs) allow administrators to control who may
+  take actions on particular queues. They are configured with the aclSubmitApps
+  and aclAdministerApps properties, which can be set per queue. Currently the
+  only supported administrative action is killing an application. Anybody who
+  may administer a queue may also submit applications to it. These properties
+  take values in a format like "user1,user2 group1,group2" or " group1,group2".
+  An action on a queue will be permitted if its user or group is in the ACL of
+  that queue or in the ACL of any of that queue's ancestors. So if queue2
+  is inside queue1, and user1 is in queue1's ACL, and user2 is in queue2's
+  ACL, then both users may submit to queue2.
+  
+  The root queue's ACLs are "*" by default which, because ACLs are passed down,
+  means that everybody may submit to and kill applications from every queue.
+  To start restricting access, change the root queue's ACLs to something other
+  than "*". 
+
   
 * {Administration}
 

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/pom.xml

@@ -119,6 +119,7 @@
     <dependency>
       <groupId>com.sun.jersey.jersey-test-framework</groupId>
       <artifactId>jersey-test-framework-grizzly2</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>com.sun.jersey</groupId>