Browse Source

Merge from trunk

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/MR-2454@1418161 13f79535-47bb-0310-9956-ffa450edef68
Alejandro Abdelnur 12 years ago
parent
commit
9975320176
100 changed files with 3351 additions and 1770 deletions
  1. 15 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 3 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java
  3. 49 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/UTF8.java
  4. 23 10
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  5. 11 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsSystem.java
  6. 46 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSinkAdapter.java
  7. 21 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
  8. 16 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
  9. 6 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java
  10. 6 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java
  11. 55 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestUTF8.java
  12. 3 26
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestGangliaMetrics.java
  13. 185 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java
  14. 153 112
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  15. 21 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java
  16. 42 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/VolumeId.java
  17. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
  18. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  19. 42 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  20. 21 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java
  21. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  22. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  23. 5 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  24. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  25. 168 157
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  26. 24 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  27. 21 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java
  28. 1 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  29. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
  30. 61 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AuditLogger.java
  31. 51 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
  32. 53 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupState.java
  33. 33 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  34. 14 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  35. 1 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  36. 173 115
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  37. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  38. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
  39. 35 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  40. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  41. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  42. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
  43. 4 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NNHAStatusHeartbeat.java
  44. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  45. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  46. 183 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestVolumeId.java
  47. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  48. 228 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java
  49. 128 23
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHFlush.java
  50. 96 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
  51. 292 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithNodeGroup.java
  52. 173 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java
  53. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
  54. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
  55. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
  56. 123 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java
  57. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
  58. 47 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
  59. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
  60. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
  61. 17 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
  62. 9 0
      hadoop-mapreduce-project/CHANGES.txt
  63. 0 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
  64. 3 24
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java
  65. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  66. 3 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskInfo.java
  67. 14 0
      hadoop-yarn-project/CHANGES.txt
  68. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestGetGroups.java
  69. 1 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  70. 0 19
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java
  71. 0 56
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/Jsons.java
  72. 0 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  73. 12 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrCompletedContainersEvent.java
  74. 65 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
  75. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
  76. 11 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
  77. 92 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/MockNodeStatusUpdater.java
  78. 222 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java
  79. 1 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
  80. 0 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
  81. 1 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
  82. 6 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
  83. 1 23
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
  84. 0 39
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ApplicationsStore.java
  85. 2 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileRMStateStore.java
  86. 0 128
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemStore.java
  87. 24 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
  88. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/Recoverable.java
  89. 0 46
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/Store.java
  90. 3 39
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/StoreFactory.java
  91. 0 509
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKStore.java
  92. 23 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/package-info.java
  93. 0 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
  94. 0 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  95. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
  96. 1 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
  97. 3 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
  98. 2 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
  99. 3 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AppSchedulable.java
  100. 88 166
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java

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

@@ -289,6 +289,9 @@ Trunk (Unreleased)
     HADOOP-9037. Bug in test-patch.sh and precommit build process (Kihwal Lee
     via jlowe)
 
+    HADOOP-9121. InodeTree.java has redundant check for vName while 
+    throwing exception. (Arup Malakar via suresh)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
@@ -306,6 +309,10 @@ Release 2.0.3-alpha - Unreleased
 
     HADOOP-9020. Add a SASL PLAIN server (daryn via bobby)
 
+    HADOOP-9090. Support on-demand publish of metrics. (Mostafa Elhemali via
+    suresh)
+
+
   IMPROVEMENTS
 
     HADOOP-8789. Tests setLevel(Level.OFF) should be Level.ERROR.
@@ -453,6 +460,14 @@ Release 2.0.3-alpha - Unreleased
     HADOOP-9064. Augment DelegationTokenRenewer API to cancel the tokens on 
     calls to removeRenewAction. (kkambatl via tucu)
 
+    HADOOP-8958. ViewFs:Non absolute mount name failures when running 
+    multiple tests on Windows. (Chris Nauroth via suresh)
+
+    HADOOP-9103. UTF8 class does not properly decode Unicode characters
+    outside the basic multilingual plane. (todd)
+
+    HADOOP-9070. Kerberos SASL server cannot find kerberos key. (daryn via atm)
+
 Release 2.0.2-alpha - 2012-09-07 
 
   INCOMPATIBLE CHANGES

+ 3 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java

@@ -118,8 +118,7 @@ abstract class InodeTree<T> {
       return result;
     }
     
-    INode<T> resolveInternal(final String pathComponent)
-        throws FileNotFoundException {
+    INode<T> resolveInternal(final String pathComponent) {
       return children.get(pathComponent);
     }
     
@@ -336,8 +335,8 @@ abstract class InodeTree<T> {
     }
     if (!gotMountTableEntry) {
       throw new IOException(
-          "ViewFs: Cannot initialize: Empty Mount table in config for " + 
-             vName == null ? "viewfs:///" : ("viewfs://" + vName + "/"));
+          "ViewFs: Cannot initialize: Empty Mount table in config for " +
+             "viewfs://" + vName + "/");
     }
   }
 

+ 49 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/UTF8.java

@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.io.DataInput;
 import java.io.DataOutput;
 
+import org.apache.hadoop.util.StringUtils;
 
 import org.apache.commons.logging.*;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -31,6 +32,9 @@ import org.apache.hadoop.classification.InterfaceStability;
  * 
  * <p>Also includes utilities for efficiently reading and writing UTF-8.
  *
+ * Note that this decodes UTF-8 but actually encodes CESU-8, a variant of
+ * UTF-8: see http://en.wikipedia.org/wiki/CESU-8
+ *
  * @deprecated replaced by Text
  */
 @Deprecated
@@ -209,6 +213,19 @@ public class UTF8 implements WritableComparable<UTF8> {
     return result;
   }
 
+  /**
+   * Convert a UTF-8 encoded byte array back into a string.
+   *
+   * @throws IOException if the byte array is invalid UTF8
+   */
+  public static String fromBytes(byte[] bytes) throws IOException {
+    DataInputBuffer dbuf = new DataInputBuffer();
+    dbuf.reset(bytes, 0, bytes.length);
+    StringBuilder buf = new StringBuilder(bytes.length);
+    readChars(dbuf, buf, bytes.length);
+    return buf.toString();
+  }
+
   /** Read a UTF-8 encoded string.
    *
    * @see DataInput#readUTF()
@@ -230,18 +247,48 @@ public class UTF8 implements WritableComparable<UTF8> {
     while (i < nBytes) {
       byte b = bytes[i++];
       if ((b & 0x80) == 0) {
+        // 0b0xxxxxxx: 1-byte sequence
         buffer.append((char)(b & 0x7F));
-      } else if ((b & 0xE0) != 0xE0) {
+      } else if ((b & 0xE0) == 0xC0) {
+        // 0b110xxxxx: 2-byte sequence
         buffer.append((char)(((b & 0x1F) << 6)
             | (bytes[i++] & 0x3F)));
-      } else {
+      } else if ((b & 0xF0) == 0xE0) {
+        // 0b1110xxxx: 3-byte sequence
         buffer.append((char)(((b & 0x0F) << 12)
             | ((bytes[i++] & 0x3F) << 6)
             |  (bytes[i++] & 0x3F)));
+      } else if ((b & 0xF8) == 0xF0) {
+        // 0b11110xxx: 4-byte sequence
+        int codepoint =
+            ((b & 0x07) << 18)
+          | ((bytes[i++] & 0x3F) <<  12)
+          | ((bytes[i++] & 0x3F) <<  6)
+          | ((bytes[i++] & 0x3F));
+        buffer.append(highSurrogate(codepoint))
+              .append(lowSurrogate(codepoint));
+      } else {
+        // The UTF8 standard describes 5-byte and 6-byte sequences, but
+        // these are no longer allowed as of 2003 (see RFC 3629)
+
+        // Only show the next 6 bytes max in the error code - in case the
+        // buffer is large, this will prevent an exceedingly large message.
+        int endForError = Math.min(i + 5, nBytes);
+        throw new IOException("Invalid UTF8 at " +
+          StringUtils.byteToHexString(bytes, i - 1, endForError));
       }
     }
   }
 
+  private static char highSurrogate(int codePoint) {
+    return (char) ((codePoint >>> 10)
+        + (Character.MIN_HIGH_SURROGATE - (Character.MIN_SUPPLEMENTARY_CODE_POINT >>> 10)));
+  }
+
+  private static char lowSurrogate(int codePoint) {
+    return (char) ((codePoint & 0x3ff) + Character.MIN_LOW_SURROGATE);
+  }
+
   /** Write a UTF-8 encoded string.
    *
    * @see DataOutput#writeUTF(String)

+ 23 - 10
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -199,7 +199,8 @@ public abstract class Server {
   //     in ObjectWritable to efficiently transmit arrays of primitives
   // 6 : Made RPC payload header explicit
   // 7 : Changed Ipc Connection Header to use Protocol buffers
-  public static final byte CURRENT_VERSION = 7;
+  // 8 : SASL server always sends a final response
+  public static final byte CURRENT_VERSION = 8;
 
   /**
    * Initial and max size of response buffer
@@ -1220,8 +1221,8 @@ public abstract class Server {
           AUDITLOG.warn(AUTH_FAILED_FOR + clientIP + ":" + attemptingUser);
           throw e;
         }
-        if (replyToken == null && authMethod == AuthMethod.PLAIN) {
-          // client needs at least response to know if it should use SIMPLE
+        if (saslServer.isComplete() && replyToken == null) {
+          // send final response for success
           replyToken = new byte[0];
         }
         if (replyToken != null) {
@@ -1392,7 +1393,7 @@ public abstract class Server {
     }
 
     private AuthMethod initializeAuthContext(AuthMethod authMethod)
-        throws IOException {
+        throws IOException, InterruptedException {
       try {
         if (enabledAuthMethods.contains(authMethod)) {
           saslServer = createSaslServer(authMethod);
@@ -1425,8 +1426,7 @@ public abstract class Server {
     }
 
     private SaslServer createSaslServer(AuthMethod authMethod)
-        throws IOException {
-      SaslServer saslServer = null;
+        throws IOException, InterruptedException {
       String hostname = null;
       String saslProtocol = null;
       CallbackHandler saslCallback = null;
@@ -1462,10 +1462,23 @@ public abstract class Server {
               "Server does not support SASL " + authMethod);
       }
       
-      String mechanism = authMethod.getMechanismName();
-      saslServer = Sasl.createSaslServer(
-          mechanism, saslProtocol, hostname,
-          SaslRpcServer.SASL_PROPS, saslCallback);
+      return createSaslServer(authMethod.getMechanismName(), saslProtocol,
+                              hostname, saslCallback);                                    
+    }
+
+    private SaslServer createSaslServer(final String mechanism,
+                                        final String protocol,
+                                        final String hostname,
+                                        final CallbackHandler callback
+        ) throws IOException, InterruptedException {
+      SaslServer saslServer = UserGroupInformation.getCurrentUser().doAs(
+          new PrivilegedExceptionAction<SaslServer>() {
+            @Override
+            public SaslServer run() throws SaslException  {
+              return Sasl.createSaslServer(mechanism, protocol, hostname,
+                                           SaslRpcServer.SASL_PROPS, callback);
+            }
+          });
       if (saslServer == null) {
         throw new AccessControlException(
             "Unable to find SASL server implementation for " + mechanism);

+ 11 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsSystem.java

@@ -90,6 +90,17 @@ public abstract class MetricsSystem implements MetricsSystemMXBean {
    */
   public abstract void register(Callback callback);
 
+  /**
+   * Requests an immediate publish of all metrics from sources to sinks.
+   * 
+   * This is a "soft" request: the expectation is that a best effort will be
+   * done to synchronously snapshot the metrics from all the sources and put
+   * them in all the sinks (including flushing the sinks) before returning to
+   * the caller. If this can't be accomplished in reasonable time it's OK to
+   * return to the caller before everything is done. 
+   */
+  public abstract void publishMetricsNow();
+
   /**
    * Shutdown the metrics system completely (usually during server shutdown.)
    * The MetricsSystemMXBean will be unregistered.

+ 46 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSinkAdapter.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.metrics2.impl;
 
 import java.util.Random;
+import java.util.concurrent.*;
 
 import static com.google.common.base.Preconditions.*;
 
@@ -48,6 +49,7 @@ class MetricsSinkAdapter implements SinkQueue.Consumer<MetricsBuffer> {
   private volatile boolean stopping = false;
   private volatile boolean inError = false;
   private final int period, firstRetryDelay, retryCount;
+  private final long oobPutTimeout;
   private final float retryBackoff;
   private final MetricsRegistry registry = new MetricsRegistry("sinkadapter");
   private final MutableStat latency;
@@ -69,6 +71,8 @@ class MetricsSinkAdapter implements SinkQueue.Consumer<MetricsBuffer> {
     this.period = checkArg(period, period > 0, "period");
     firstRetryDelay = checkArg(retryDelay, retryDelay > 0, "retry delay");
     this.retryBackoff = checkArg(retryBackoff, retryBackoff>1, "retry backoff");
+    oobPutTimeout = (long)
+        (firstRetryDelay * Math.pow(retryBackoff, retryCount) * 1000);
     this.retryCount = retryCount;
     this.queue = new SinkQueue<MetricsBuffer>(checkArg(queueCapacity,
         queueCapacity > 0, "queue capacity"));
@@ -95,6 +99,23 @@ class MetricsSinkAdapter implements SinkQueue.Consumer<MetricsBuffer> {
     }
     return true; // OK
   }
+  
+  public boolean putMetricsImmediate(MetricsBuffer buffer) {
+    WaitableMetricsBuffer waitableBuffer =
+        new WaitableMetricsBuffer(buffer);
+    if (!queue.enqueue(waitableBuffer)) {
+      LOG.warn(name + " has a full queue and can't consume the given metrics.");
+      dropped.incr();
+      return false;
+    }
+    if (!waitableBuffer.waitTillNotified(oobPutTimeout)) {
+      LOG.warn(name +
+          " couldn't fulfill an immediate putMetrics request in time." +
+          " Abandoning.");
+      return false;
+    }
+    return true;
+  }
 
   void publishMetricsFromQueue() {
     int retryDelay = firstRetryDelay;
@@ -158,6 +179,9 @@ class MetricsSinkAdapter implements SinkQueue.Consumer<MetricsBuffer> {
       sink.flush();
       latency.add(Time.now() - ts);
     }
+    if (buffer instanceof WaitableMetricsBuffer) {
+      ((WaitableMetricsBuffer)buffer).notifyAnyWaiters();
+    }
     LOG.debug("Done");
   }
 
@@ -191,4 +215,26 @@ class MetricsSinkAdapter implements SinkQueue.Consumer<MetricsBuffer> {
   MetricsSink sink() {
     return sink;
   }
+
+  static class WaitableMetricsBuffer extends MetricsBuffer {
+    private final Semaphore notificationSemaphore =
+        new Semaphore(0);
+
+    public WaitableMetricsBuffer(MetricsBuffer metricsBuffer) {
+      super(metricsBuffer);
+    }
+
+    public boolean waitTillNotified(long millisecondsToWait) {
+      try {
+        return notificationSemaphore.tryAcquire(millisecondsToWait,
+            TimeUnit.MILLISECONDS);
+      } catch (InterruptedException e) {
+        return false;
+      }
+    }
+
+    public void notifyAnyWaiters() {
+      notificationSemaphore.release();
+    }
+  }
 }

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

@@ -344,9 +344,19 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
   synchronized void onTimerEvent() {
     logicalTime += period;
     if (sinks.size() > 0) {
-      publishMetrics(sampleMetrics());
+      publishMetrics(sampleMetrics(), false);
     }
   }
+  
+  /**
+   * Requests an immediate publish of all metrics from sources to sinks.
+   */
+  @Override
+  public void publishMetricsNow() {
+    if (sinks.size() > 0) {
+      publishMetrics(sampleMetrics(), true);
+    }    
+  }
 
   /**
    * Sample all the sources for a snapshot of metrics/tags
@@ -380,12 +390,20 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
   /**
    * Publish a metrics snapshot to all the sinks
    * @param buffer  the metrics snapshot to publish
+   * @param immediate  indicates that we should publish metrics immediately
+   *                   instead of using a separate thread.
    */
-  synchronized void publishMetrics(MetricsBuffer buffer) {
+  synchronized void publishMetrics(MetricsBuffer buffer, boolean immediate) {
     int dropped = 0;
     for (MetricsSinkAdapter sa : sinks.values()) {
       long startTime = Time.now();
-      dropped += sa.putMetrics(buffer, logicalTime) ? 0 : 1;
+      boolean result;
+      if (immediate) {
+        result = sa.putMetricsImmediate(buffer); 
+      } else {
+        result = sa.putMetrics(buffer, logicalTime);
+      }
+      dropped += result ? 0 : 1;
       publishStat.add(Time.now() - startTime);
     }
     droppedPubAll.incr(dropped);

+ 16 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java

@@ -28,6 +28,9 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.util.ReflectionUtils;
 
 /** The class represents a cluster of computer with a tree hierarchical
  * network topology.
@@ -52,6 +55,19 @@ public class NetworkTopology {
       super(msg);
     }
   }
+  
+  /**
+   * Get an instance of NetworkTopology based on the value of the configuration
+   * parameter net.topology.impl.
+   * 
+   * @param conf the configuration to be used
+   * @return an instance of NetworkTopology
+   */
+  public static NetworkTopology getInstance(Configuration conf){
+    return ReflectionUtils.newInstance(
+        conf.getClass(CommonConfigurationKeysPublic.NET_TOPOLOGY_IMPL_KEY,
+        NetworkTopology.class, NetworkTopology.class), conf);
+  }
 
   /** InnerNode represents a switch/router of a data center or rack.
    * Different from a leaf node, it has non-null children.

+ 6 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.viewfs.ConfigUtil;
+import org.apache.hadoop.util.Shell;
 import org.mortbay.log.Log;
 
 
@@ -123,8 +124,11 @@ public class ViewFileSystemTestSetup {
    * in the target file system.
    */
   static void linkUpFirstComponents(Configuration conf, String path, FileSystem fsTarget, String info) {
-    int indexOf2ndSlash = path.indexOf('/', 1);
-    String firstComponent = path.substring(0, indexOf2ndSlash);
+    int indexOfEnd = path.indexOf('/', 1);
+    if (Shell.WINDOWS) {
+      indexOfEnd = path.indexOf('/', indexOfEnd + 1);
+    }
+    String firstComponent = path.substring(0, indexOfEnd);
     URI linkTarget = fsTarget.makeQualified(new Path(firstComponent)).toUri();
     ConfigUtil.addLink(conf, firstComponent, linkTarget);
     Log.info("Added link for " + info + " " 

+ 6 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsTestSetup.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.FileContextTestHelper;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.viewfs.ConfigUtil;
+import org.apache.hadoop.util.Shell;
 import org.mortbay.log.Log;
 
 
@@ -120,8 +121,11 @@ public class ViewFsTestSetup {
    */
   static void linkUpFirstComponents(Configuration conf, String path,
       FileContext fsTarget, String info) {
-    int indexOf2ndSlash = path.indexOf('/', 1);
-    String firstComponent = path.substring(0, indexOf2ndSlash);
+    int indexOfEnd = path.indexOf('/', 1);
+    if (Shell.WINDOWS) {
+      indexOfEnd = path.indexOf('/', indexOfEnd + 1);
+    }
+    String firstComponent = path.substring(0, indexOfEnd);
     URI linkTarget = fsTarget.makeQualified(new Path(firstComponent)).toUri();
     ConfigUtil.addLink(conf, firstComponent, linkTarget);
     Log.info("Added link for " + info + " " 

+ 55 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestUTF8.java

@@ -19,8 +19,12 @@
 package org.apache.hadoop.io;
 
 import junit.framework.TestCase;
+import java.io.IOException;
 import java.util.Random;
 
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.StringUtils;
+
 /** Unit tests for UTF8. */
 @SuppressWarnings("deprecation")
 public class TestUTF8 extends TestCase {
@@ -92,5 +96,55 @@ public class TestUTF8 extends TestCase {
 
     assertEquals(s, new String(dob.getData(), 2, dob.getLength()-2, "UTF-8"));
   }
-	
+
+  /**
+   * Test encoding and decoding of UTF8 outside the basic multilingual plane.
+   *
+   * This is a regression test for HADOOP-9103.
+   */
+  public void testNonBasicMultilingualPlane() throws Exception {
+    // Test using the "CAT FACE" character (U+1F431)
+    // See http://www.fileformat.info/info/unicode/char/1f431/index.htm
+    String catFace = "\uD83D\uDC31";
+
+    // This encodes to 4 bytes in UTF-8:
+    byte[] encoded = catFace.getBytes("UTF-8");
+    assertEquals(4, encoded.length);
+    assertEquals("f09f90b1", StringUtils.byteToHexString(encoded));
+
+    // Decode back to String using our own decoder
+    String roundTrip = UTF8.fromBytes(encoded);
+    assertEquals(catFace, roundTrip);
+  }
+
+  /**
+   * Test that decoding invalid UTF8 throws an appropriate error message.
+   */
+  public void testInvalidUTF8() throws Exception {
+    byte[] invalid = new byte[] {
+        0x01, 0x02, (byte)0xff, (byte)0xff, 0x01, 0x02, 0x03, 0x04, 0x05 };
+    try {
+      UTF8.fromBytes(invalid);
+      fail("did not throw an exception");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains(
+          "Invalid UTF8 at ffff01020304", ioe);
+    }
+  }
+
+  /**
+   * Test for a 5-byte UTF8 sequence, which is now considered illegal.
+   */
+  public void test5ByteUtf8Sequence() throws Exception {
+    byte[] invalid = new byte[] {
+        0x01, 0x02, (byte)0xf8, (byte)0x88, (byte)0x80,
+        (byte)0x80, (byte)0x80, 0x04, 0x05 };
+    try {
+      UTF8.fromBytes(invalid);
+      fail("did not throw an exception");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains(
+          "Invalid UTF8 at f88880808004", ioe);
+    }
+  }
 }

+ 3 - 26
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestGangliaMetrics.java

@@ -29,8 +29,6 @@ import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -115,31 +113,23 @@ public class TestGangliaMetrics {
     final int expectedCountFromGanglia30 = expectedMetrics.length;
     final int expectedCountFromGanglia31 = 2 * expectedMetrics.length;
 
-    // use latch to make sure we received required records before shutting
-    // down the MetricSystem
-    CountDownLatch latch = new CountDownLatch(
-        expectedCountFromGanglia30 + expectedCountFromGanglia31);
-
     // Setup test for GangliaSink30
     AbstractGangliaSink gsink30 = new GangliaSink30();
     gsink30.init(cb.subset("test"));
-    MockDatagramSocket mockds30 = new MockDatagramSocket(latch);
+    MockDatagramSocket mockds30 = new MockDatagramSocket();
     GangliaMetricsTestHelper.setDatagramSocket(gsink30, mockds30);
 
     // Setup test for GangliaSink31
     AbstractGangliaSink gsink31 = new GangliaSink31();
     gsink31.init(cb.subset("test"));
-    MockDatagramSocket mockds31 = new MockDatagramSocket(latch);
+    MockDatagramSocket mockds31 = new MockDatagramSocket();
     GangliaMetricsTestHelper.setDatagramSocket(gsink31, mockds31);
 
     // register the sinks
     ms.register("gsink30", "gsink30 desc", gsink30);
     ms.register("gsink31", "gsink31 desc", gsink31);
-    ms.onTimerEvent();  // trigger something interesting
+    ms.publishMetricsNow(); // publish the metrics
 
-    // wait for all records and the stop MetricSystem.  Without this
-    // sometime the ms gets shutdown before all the sinks have consumed
-    latch.await(200, TimeUnit.MILLISECONDS);
     ms.stop();
 
     // check GanfliaSink30 data
@@ -198,7 +188,6 @@ public class TestGangliaMetrics {
    */
   private class MockDatagramSocket extends DatagramSocket {
     private ArrayList<byte[]> capture;
-    private CountDownLatch latch;
 
     /**
      * @throws SocketException
@@ -207,15 +196,6 @@ public class TestGangliaMetrics {
       capture = new  ArrayList<byte[]>();
     }
 
-    /**
-     * @param latch
-     * @throws SocketException
-     */
-    public MockDatagramSocket(CountDownLatch latch) throws SocketException {
-      this();
-      this.latch = latch;
-    }
-
     /* (non-Javadoc)
      * @see java.net.DatagramSocket#send(java.net.DatagramPacket)
      */
@@ -225,9 +205,6 @@ public class TestGangliaMetrics {
       byte[] bytes = new byte[p.getLength()];
       System.arraycopy(p.getData(), p.getOffset(), bytes, 0, p.getLength());
       capture.add(bytes);
-
-      // decrement the latch
-      latch.countDown();
     }
 
     /**

+ 185 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestMetricsSystemImpl.java

@@ -18,7 +18,11 @@
 
 package org.apache.hadoop.metrics2.impl;
 
-import java.util.List;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import javax.annotation.Nullable;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -26,9 +30,11 @@ import org.junit.runner.RunWith;
 import org.mockito.ArgumentCaptor;
 import org.mockito.Captor;
 import org.mockito.runners.MockitoJUnitRunner;
+
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 
+import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 
 import org.apache.commons.configuration.SubsetConfiguration;
@@ -36,6 +42,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.metrics2.MetricsException;
 import static org.apache.hadoop.test.MoreAsserts.*;
+
+import org.apache.hadoop.metrics2.AbstractMetric;
 import org.apache.hadoop.metrics2.MetricsRecord;
 import org.apache.hadoop.metrics2.MetricsSink;
 import org.apache.hadoop.metrics2.MetricsSource;
@@ -47,6 +55,7 @@ import org.apache.hadoop.metrics2.lib.MetricsRegistry;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
 import org.apache.hadoop.metrics2.lib.MutableRate;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
+import org.apache.hadoop.util.StringUtils;
 
 /**
  * Test the MetricsSystemImpl class
@@ -72,7 +81,7 @@ public class TestMetricsSystemImpl {
   }
 
   @Test public void testInitFirst() throws Exception {
-    ConfigBuilder cb = new ConfigBuilder().add("*.period", 8)
+    new ConfigBuilder().add("*.period", 8)
         //.add("test.sink.plugin.urls", getPluginUrlsAsString())
         .add("test.sink.test.class", TestSink.class.getName())
         .add("test.*.source.filter.exclude", "s0")
@@ -93,8 +102,9 @@ public class TestMetricsSystemImpl {
     MetricsSink sink2 = mock(MetricsSink.class);
     ms.registerSink("sink1", "sink1 desc", sink1);
     ms.registerSink("sink2", "sink2 desc", sink2);
-    ms.onTimerEvent();  // trigger something interesting
+    ms.publishMetricsNow(); // publish the metrics
     ms.stop();
+    ms.shutdown();
 
     verify(sink1, times(2)).putMetrics(r1.capture());
     List<MetricsRecord> mr1 = r1.getAllValues();
@@ -104,6 +114,177 @@ public class TestMetricsSystemImpl {
     assertEquals("output", mr1, mr2);
   }
 
+  @Test public void testMultiThreadedPublish() throws Exception {
+    new ConfigBuilder().add("*.period", 80)
+      .add("test.sink.Collector.queue.capacity", "20")
+      .save(TestMetricsConfig.getTestFilename("hadoop-metrics2-test"));
+    final MetricsSystemImpl ms = new MetricsSystemImpl("Test");
+    ms.start();
+    final int numThreads = 10;
+    final CollectingSink sink = new CollectingSink(numThreads);
+    ms.registerSink("Collector",
+        "Collector of values from all threads.", sink);
+    final TestSource[] sources = new TestSource[numThreads];
+    final Thread[] threads = new Thread[numThreads];
+    final String[] results = new String[numThreads];
+    final CyclicBarrier barrier1 = new CyclicBarrier(numThreads),
+        barrier2 = new CyclicBarrier(numThreads);
+    for (int i = 0; i < numThreads; i++) {
+      sources[i] = ms.register("threadSource" + i,
+          "A source of my threaded goodness.",
+          new TestSource("threadSourceRec" + i));
+      threads[i] = new Thread(new Runnable() {
+        private boolean safeAwait(int mySource, CyclicBarrier barrier) {
+          try {
+            barrier1.await(2, TimeUnit.SECONDS);
+          } catch (InterruptedException e) {
+            results[mySource] = "Interrupted";
+            return false;
+          } catch (BrokenBarrierException e) {
+            results[mySource] = "Broken Barrier";
+            return false;
+          } catch (TimeoutException e) {
+            results[mySource] = "Timed out on barrier";
+            return false;
+          }
+          return true;
+        }
+        
+        @Override
+        public void run() {
+          int mySource = Integer.parseInt(Thread.currentThread().getName());
+          if (sink.collected[mySource].get() != 0L) {
+            results[mySource] = "Someone else collected my metric!";
+            return;
+          }
+          // Wait for all the threads to come here so we can hammer
+          // the system at the same time
+          if (!safeAwait(mySource, barrier1)) return;
+          sources[mySource].g1.set(230);
+          ms.publishMetricsNow();
+          // Since some other thread may have snatched my metric,
+          // I need to wait for the threads to finish before checking.
+          if (!safeAwait(mySource, barrier2)) return;
+          if (sink.collected[mySource].get() != 230L) {
+            results[mySource] = "Metric not collected!";
+            return;
+          }
+          results[mySource] = "Passed";
+        }
+      }, "" + i);
+    }
+    for (Thread t : threads)
+      t.start();
+    for (Thread t : threads)
+      t.join();
+    assertEquals(0L, ms.droppedPubAll.value());
+    assertTrue(StringUtils.join("\n", Arrays.asList(results)),
+      Iterables.all(Arrays.asList(results), new Predicate<String>() {
+        @Override
+        public boolean apply(@Nullable String input) {
+          return input.equalsIgnoreCase("Passed");
+        }
+      }));
+    ms.stop();
+    ms.shutdown();
+  }
+
+  private static class CollectingSink implements MetricsSink {
+    private final AtomicLong[] collected;
+    
+    public CollectingSink(int capacity) {
+      collected = new AtomicLong[capacity];
+      for (int i = 0; i < capacity; i++) {
+        collected[i] = new AtomicLong();
+      }
+    }
+    
+    @Override
+    public void init(SubsetConfiguration conf) {
+    }
+
+    @Override
+    public void putMetrics(MetricsRecord record) {
+      final String prefix = "threadSourceRec";
+      if (record.name().startsWith(prefix)) {
+        final int recordNumber = Integer.parseInt(
+            record.name().substring(prefix.length()));
+        ArrayList<String> names = new ArrayList<String>();
+        for (AbstractMetric m : record.metrics()) {
+          if (m.name().equalsIgnoreCase("g1")) {
+            collected[recordNumber].set(m.value().longValue());
+            return;
+          }
+          names.add(m.name());
+        }
+      }
+    }
+
+    @Override
+    public void flush() {
+    }
+  }
+
+  @Test public void testHangingSink() {
+    new ConfigBuilder().add("*.period", 8)
+      .add("test.sink.test.class", TestSink.class.getName())
+      .add("test.sink.hanging.retry.delay", "1")
+      .add("test.sink.hanging.retry.backoff", "1.01")
+      .add("test.sink.hanging.retry.count", "0")
+      .save(TestMetricsConfig.getTestFilename("hadoop-metrics2-test"));
+    MetricsSystemImpl ms = new MetricsSystemImpl("Test");
+    ms.start();
+    TestSource s = ms.register("s3", "s3 desc", new TestSource("s3rec"));
+    s.c1.incr();
+    HangingSink hanging = new HangingSink();
+    ms.registerSink("hanging", "Hang the sink!", hanging);
+    ms.publishMetricsNow();
+    assertEquals(1L, ms.droppedPubAll.value());
+    assertFalse(hanging.getInterrupted());
+    ms.stop();
+    ms.shutdown();
+    assertTrue(hanging.getInterrupted());
+    assertTrue("The sink didn't get called after its first hang " +
+               "for subsequent records.", hanging.getGotCalledSecondTime());
+  }
+
+  private static class HangingSink implements MetricsSink {
+    private volatile boolean interrupted;
+    private boolean gotCalledSecondTime;
+    private boolean firstTime = true;
+
+    public boolean getGotCalledSecondTime() {
+      return gotCalledSecondTime;
+    }
+
+    public boolean getInterrupted() {
+      return interrupted;
+    }
+
+    @Override
+    public void init(SubsetConfiguration conf) {
+    }
+
+    @Override
+    public void putMetrics(MetricsRecord record) {
+      // No need to hang every time, just the first record.
+      if (!firstTime) {
+        gotCalledSecondTime = true;
+        return;
+      }
+      firstTime = false;
+      try {
+        Thread.sleep(10 * 1000);
+      } catch (InterruptedException ex) {
+        interrupted = true;
+      }
+    }
+
+    @Override
+    public void flush() {
+    }
+  }
+
   @Test public void testRegisterDups() {
     MetricsSystem ms = new MetricsSystemImpl();
     TestSource ts1 = new TestSource("ts1");
@@ -116,6 +297,7 @@ public class TestMetricsSystemImpl {
     MetricsSource s2 = ms.getSource("ts1");
     assertNotNull(s2);
     assertNotSame(s1, s2);
+    ms.shutdown();
   }
 
   @Test(expected=MetricsException.class) public void testRegisterDupError() {

+ 153 - 112
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -17,6 +17,9 @@ Trunk (Unreleased)
     reliably storing HDFS edit logs. See dedicated section below for breakdown
     of subtasks.
 
+    HDFS-3495. Update Balancer to support new NetworkTopology with NodeGroup.
+    (Junping Du via szetszwo)
+
   IMPROVEMENTS
 
     HDFS-1620. Rename HdfsConstants -> HdfsServerConstants, FSConstants ->
@@ -89,17 +92,12 @@ Trunk (Unreleased)
     HDFS-3040. TestMulitipleNNDataBlockScanner is misspelled. (Madhukara Phatak
     via atm)
 
-    HDFS-3049. During the normal NN startup process, fall back on a different
-    edit log if we see one that is corrupt (Colin Patrick McCabe via todd)
-
     HDFS-3478. Test quotas with Long.Max_Value. (Sujay Rau via eli)
 
     HDFS-3498. Support replica removal in BlockPlacementPolicy and make
     BlockPlacementPolicyDefault extensible for reusing code in subclasses.
     (Junping Du via szetszwo)
 
-    HDFS-3571. Allow EditLogFileInputStream to read from a remote URL (todd)
-
     HDFS-3510.  Editlog pre-allocation is performed prior to writing edits
     to avoid partial edits case disk out of space.(Colin McCabe via suresh)
 
@@ -143,8 +141,6 @@ Trunk (Unreleased)
     HDFS-4052. BlockManager#invalidateWork should print log outside the lock.
     (Jing Zhao via suresh)
 
-    HDFS-4110. Refine a log printed in JNStorage. (Liang Xie via suresh)
-
     HDFS-4124. Refactor INodeDirectory#getExistingPathINodes() to enable 
     returning more than INode array. (Jing Zhao via suresh)
 
@@ -157,10 +153,6 @@ Trunk (Unreleased)
     HDFS-4152. Add a new class BlocksMapUpdateInfo for the parameter in
     INode.collectSubtreeBlocksAndClear(..). (Jing Zhao via szetszwo)
 
-    HDFS-4153. Add START_MSG/SHUTDOWN_MSG for JournalNode. (liang xie via atm)
-
-    HDFS-3935. Add JournalNode to the start/stop scripts (Andy Isaacson via todd)
-
     HDFS-4206. Change the fields in INode and its subclasses to private.
     (szetszwo)
 
@@ -173,6 +165,11 @@ Trunk (Unreleased)
     HDFS-4209. Clean up the addNode/addChild/addChildNoQuotaCheck methods in
     FSDirectory and INodeDirectory. (szetszwo)
 
+    HDFS-3358. Specify explicitly that the NN UI status total is talking
+    of persistent objects on heap. (harsh)
+
+    HDFS-4234. Use generic code for choosing datanode in Balancer.  (szetszwo)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -265,107 +262,9 @@ Trunk (Unreleased)
     HDFS-4105. The SPNEGO user for secondary namenode should use the web 
     keytab. (Arpit Gupta via jitendra)
 
-  BREAKDOWN OF HDFS-3077 SUBTASKS
-
-    HDFS-3077. Quorum-based protocol for reading and writing edit logs.
-    (todd, Brandon Li, and Hari Mankude via todd)
-    
-    HDFS-3694. Fix getEditLogManifest to fetch httpPort if necessary (todd)
-    
-    HDFS-3692. Support purgeEditLogs() call to remotely purge logs on JNs
-    (todd)
-    
-    HDFS-3693. JNStorage should read its storage info even before a writer
-    becomes active (todd)
-    
-    HDFS-3725. Fix QJM startup when individual JNs have gaps (todd)
-    
-    HDFS-3741. Exhaustive failure injection test for skipped RPCs (todd)
-    
-    HDFS-3773. TestNNWithQJM fails after HDFS-3741. (atm)
-    
-    HDFS-3793. Implement genericized format() in QJM (todd)
-    
-    HDFS-3795. QJM: validate journal dir at startup (todd)
-    
-    HDFS-3798. Avoid throwing NPE when finalizeSegment() is called on invalid
-    segment (todd)
-    
-    HDFS-3799. QJM: handle empty log segments during recovery (todd)
-    
-    HDFS-3797. QJM: add segment txid as a parameter to journal() RPC (todd)
-    
-    HDFS-3800. improvements to QJM fault testing (todd)
-    
-    HDFS-3823. QJM: TestQJMWithFaults fails occasionally because of missed
-    setting of HTTP port. (todd and atm)
-    
-    HDFS-3826. QJM: Some trivial logging / exception text improvements. (todd
-    and atm)
-    
-    HDFS-3839. QJM: hadoop-daemon.sh should be updated to accept "journalnode"
-    (eli)
-    
-    HDFS-3845. Fixes for edge cases in QJM recovery protocol (todd)
-    
-    HDFS-3877. QJM: Provide defaults for dfs.journalnode.*address (eli)
-    
-    HDFS-3863. Track last "committed" txid in QJM (todd)
-    
-    HDFS-3869. Expose non-file journal manager details in web UI (todd)
-    
-    HDFS-3884. Journal format() should reset cached values (todd)
-    
-    HDFS-3870. Add metrics to JournalNode (todd)
-    
-    HDFS-3891. Make selectInputStreams throw IOE instead of RTE (todd)
-    
-    HDFS-3726. If a logger misses an RPC, don't retry that logger until next
-    segment (todd)
-    
-    HDFS-3893. QJM: Make QJM work with security enabled. (atm)
-    
-    HDFS-3897. QJM: TestBlockToken fails after HDFS-3893. (atm)
-    
-    HDFS-3898. QJM: enable TCP_NODELAY for IPC (todd)
-    
-    HDFS-3885. QJM: optimize log sync when JN is lagging behind (todd)
-    
-    HDFS-3900. QJM: avoid validating log segments on log rolls (todd)
-    
-    HDFS-3901. QJM: send 'heartbeat' messages to JNs even when they are
-    out-of-sync (todd)
-    
-    HDFS-3899. QJM: Add client-side metrics (todd)
-    
-    HDFS-3914. QJM: acceptRecovery should abort current segment (todd)
-    
-    HDFS-3915. QJM: Failover fails with auth error in secure cluster (todd)
-    
-    HDFS-3906. QJM: quorum timeout on failover with large log segment (todd)
-    
-    HDFS-3840. JournalNodes log JournalNotFormattedException backtrace error
-    before being formatted (todd)
-    
-    HDFS-3894. QJM: testRecoverAfterDoubleFailures can be flaky due to IPC
-    client caching (todd)
-    
-    HDFS-3926. QJM: Add user documentation for QJM. (atm)
-    
-    HDFS-3943. QJM: remove currently-unused md5sum field (todd)
-    
-    HDFS-3950. QJM: misc TODO cleanup, improved log messages, etc. (todd)
-    
-    HDFS-3955. QJM: Make acceptRecovery() atomic. (todd)
-    
-    HDFS-3956. QJM: purge temporary files when no longer within retention
-    period (todd)
-    
-    HDFS-4004. TestJournalNode#testJournal fails because of test case execution
-    order (Chao Shi via todd)
-    
-    HDFS-4017. Unclosed FileInputStream in GetJournalEditServlet
-    (Chao Shi via todd)
+    HDFS-4240. For nodegroup-aware block placement, when a node is excluded,
+    the nodes in the same nodegroup should also be excluded.  (Junping Du
+    via szetszwo)
 
 Release 2.0.3-alpha - Unreleased 
 
@@ -386,6 +285,9 @@ Release 2.0.3-alpha - Unreleased
 
     HDFS-4155. libhdfs implementation of hsync API (Liang Xie via todd)
 
+    HDFS-4213. Add an API to hsync for updating the last block length at the
+    namenode. (Jing Zhao via szetszwo)
+
   IMPROVEMENTS
   
     HDFS-3925. Prettify PipelineAck#toString() for printing to a log
@@ -487,6 +389,24 @@ Release 2.0.3-alpha - Unreleased
     HDFS-4214. OfflineEditsViewer should print out the offset at which it
     encountered an error. (Colin Patrick McCabe via atm)
 
+    HDFS-4199. Provide test for HdfsVolumeId. (Ivan A. Veselovsky via atm)
+
+    HDFS-3049. During the normal NN startup process, fall back on a different
+    edit log if we see one that is corrupt (Colin Patrick McCabe via todd)
+
+    HDFS-3571. Allow EditLogFileInputStream to read from a remote URL (todd)
+
+    HDFS-4110. Refine a log printed in JNStorage. (Liang Xie via suresh)
+
+    HDFS-4153. Add START_MSG/SHUTDOWN_MSG for JournalNode. (liang xie via atm)
+
+    HDFS-3935. Add JournalNode to the start/stop scripts (Andy Isaacson via todd)
+
+    HDFS-4268. Remove redundant enum NNHAStatusHeartbeat.State. (shv)
+
+    HDFS-3680. Allow customized audit logging in HDFS FSNamesystem. (Marcelo
+    Vanzin via atm)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -636,6 +556,122 @@ Release 2.0.3-alpha - Unreleased
     HDFS-4216. Do not ignore QuotaExceededException when adding symlinks.
     (szetszwo)
 
+    HDFS-4242. Map.Entry is incorrectly used in LeaseManager since the behavior
+    of it is undefined after the iteration or modifications of the map.
+    (szetszwo)
+
+    HDFS-4231. BackupNode: Introduce BackupState. (shv)
+
+    HDFS-4243. When replacing an INodeDirectory, the parent pointers of the
+    children of the child have to be updated to the new child.  (Jing Zhao
+    via szetszwo)
+
+    HDFS-4238. Standby namenode should not do purging of shared
+    storage edits. (todd)
+
+  BREAKDOWN OF HDFS-3077 SUBTASKS
+
+    HDFS-3077. Quorum-based protocol for reading and writing edit logs.
+    (todd, Brandon Li, and Hari Mankude via todd)
+    
+    HDFS-3694. Fix getEditLogManifest to fetch httpPort if necessary (todd)
+    
+    HDFS-3692. Support purgeEditLogs() call to remotely purge logs on JNs
+    (todd)
+    
+    HDFS-3693. JNStorage should read its storage info even before a writer
+    becomes active (todd)
+    
+    HDFS-3725. Fix QJM startup when individual JNs have gaps (todd)
+    
+    HDFS-3741. Exhaustive failure injection test for skipped RPCs (todd)
+    
+    HDFS-3773. TestNNWithQJM fails after HDFS-3741. (atm)
+    
+    HDFS-3793. Implement genericized format() in QJM (todd)
+    
+    HDFS-3795. QJM: validate journal dir at startup (todd)
+    
+    HDFS-3798. Avoid throwing NPE when finalizeSegment() is called on invalid
+    segment (todd)
+    
+    HDFS-3799. QJM: handle empty log segments during recovery (todd)
+    
+    HDFS-3797. QJM: add segment txid as a parameter to journal() RPC (todd)
+    
+    HDFS-3800. improvements to QJM fault testing (todd)
+    
+    HDFS-3823. QJM: TestQJMWithFaults fails occasionally because of missed
+    setting of HTTP port. (todd and atm)
+    
+    HDFS-3826. QJM: Some trivial logging / exception text improvements. (todd
+    and atm)
+    
+    HDFS-3839. QJM: hadoop-daemon.sh should be updated to accept "journalnode"
+    (eli)
+    
+    HDFS-3845. Fixes for edge cases in QJM recovery protocol (todd)
+    
+    HDFS-3877. QJM: Provide defaults for dfs.journalnode.*address (eli)
+    
+    HDFS-3863. Track last "committed" txid in QJM (todd)
+    
+    HDFS-3869. Expose non-file journal manager details in web UI (todd)
+    
+    HDFS-3884. Journal format() should reset cached values (todd)
+    
+    HDFS-3870. Add metrics to JournalNode (todd)
+    
+    HDFS-3891. Make selectInputStreams throw IOE instead of RTE (todd)
+    
+    HDFS-3726. If a logger misses an RPC, don't retry that logger until next
+    segment (todd)
+    
+    HDFS-3893. QJM: Make QJM work with security enabled. (atm)
+    
+    HDFS-3897. QJM: TestBlockToken fails after HDFS-3893. (atm)
+    
+    HDFS-3898. QJM: enable TCP_NODELAY for IPC (todd)
+    
+    HDFS-3885. QJM: optimize log sync when JN is lagging behind (todd)
+    
+    HDFS-3900. QJM: avoid validating log segments on log rolls (todd)
+    
+    HDFS-3901. QJM: send 'heartbeat' messages to JNs even when they are
+    out-of-sync (todd)
+    
+    HDFS-3899. QJM: Add client-side metrics (todd)
+    
+    HDFS-3914. QJM: acceptRecovery should abort current segment (todd)
+    
+    HDFS-3915. QJM: Failover fails with auth error in secure cluster (todd)
+    
+    HDFS-3906. QJM: quorum timeout on failover with large log segment (todd)
+    
+    HDFS-3840. JournalNodes log JournalNotFormattedException backtrace error
+    before being formatted (todd)
+    
+    HDFS-3894. QJM: testRecoverAfterDoubleFailures can be flaky due to IPC
+    client caching (todd)
+    
+    HDFS-3926. QJM: Add user documentation for QJM. (atm)
+    
+    HDFS-3943. QJM: remove currently-unused md5sum field (todd)
+    
+    HDFS-3950. QJM: misc TODO cleanup, improved log messages, etc. (todd)
+    
+    HDFS-3955. QJM: Make acceptRecovery() atomic. (todd)
+    
+    HDFS-3956. QJM: purge temporary files when no longer within retention
+    period (todd)
+    
+    HDFS-4004. TestJournalNode#testJournal fails because of test case execution
+    order (Chao Shi via todd)
+    
+    HDFS-4017. Unclosed FileInputStream in GetJournalEditServlet
+    (Chao Shi via todd)
+
+
 Release 2.0.2-alpha - 2012-09-07 
 
   INCOMPATIBLE CHANGES
@@ -2025,6 +2061,11 @@ Release 0.23.6 - UNRELEASED
 
   BUG FIXES
 
+    HDFS-4247. saveNamespace should be tolerant of dangling lease (daryn)
+
+    HDFS-4248. Renaming directories may incorrectly remove the paths in leases
+    under the tree.  (daryn via szetszwo)
+
 Release 0.23.5 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 21 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java

@@ -27,26 +27,38 @@ import org.apache.hadoop.classification.InterfaceStability;
  * HDFS-specific volume identifier which implements {@link VolumeId}. Can be
  * used to differentiate between the data directories on a single datanode. This
  * identifier is only unique on a per-datanode basis.
+ * 
+ * Note that invalid IDs are represented by {@link VolumeId#INVALID_VOLUME_ID}.
  */
 @InterfaceStability.Unstable
 @InterfaceAudience.Public
 public class HdfsVolumeId implements VolumeId {
-
+  
   private final byte[] id;
-  private final boolean isValid;
 
-  public HdfsVolumeId(byte[] id, boolean isValid) {
+  public HdfsVolumeId(byte[] id) {
+    if (id == null) {
+      throw new NullPointerException("A valid Id can only be constructed " +
+      		"with a non-null byte array.");
+    }
     this.id = id;
-    this.isValid = isValid;
   }
 
   @Override
-  public boolean isValid() {
-    return isValid;
+  public final boolean isValid() {
+    return true;
   }
 
   @Override
   public int compareTo(VolumeId arg0) {
+    if (arg0 == null) {
+      return 1;
+    }
+    if (!arg0.isValid()) {
+      // any valid ID is greater 
+      // than any invalid ID: 
+      return 1;
+    }
     return hashCode() - arg0.hashCode();
   }
 
@@ -63,8 +75,10 @@ public class HdfsVolumeId implements VolumeId {
     if (obj == this) {
       return true;
     }
-
     HdfsVolumeId that = (HdfsVolumeId) obj;
+    // NB: if (!obj.isValid()) { return false; } check is not necessary
+    // because we have class identity checking above, and for this class
+    // isValid() is always true.
     return new EqualsBuilder().append(this.id, that.id).isEquals();
   }
 

+ 42 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/VolumeId.java

@@ -28,6 +28,48 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceAudience.Public
 public interface VolumeId extends Comparable<VolumeId> {
 
+  /**
+   * Represents an invalid Volume ID (ID for unknown content).
+   */
+  public static final VolumeId INVALID_VOLUME_ID = new VolumeId() {
+    
+    @Override
+    public int compareTo(VolumeId arg0) {
+      // This object is equal only to itself;
+      // It is greater than null, and
+      // is always less than any other VolumeId:
+      if (arg0 == null) {
+        return 1;
+      }
+      if (arg0 == this) {
+        return 0;
+      } else {
+        return -1;
+      }
+    }
+    
+    @Override
+    public boolean equals(Object obj) {
+      // this object is equal only to itself:
+      return (obj == this);
+    }
+    
+    @Override
+    public int hashCode() {
+      return Integer.MIN_VALUE;
+    }
+    
+    @Override
+    public boolean isValid() {
+      return false;
+    }
+    
+    @Override
+    public String toString() {
+      return "Invalid VolumeId";
+    }
+  };
+  
   /**
    * Indicates if the disk identifier is valid. Invalid identifiers indicate
    * that the block was not present, or the location could otherwise not be

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

@@ -202,7 +202,7 @@ class BlockStorageLocationUtil {
       ArrayList<VolumeId> l = new ArrayList<VolumeId>(b.getLocations().length);
       // Start off all IDs as invalid, fill it in later with results from RPCs
       for (int i = 0; i < b.getLocations().length; i++) {
-        l.add(new HdfsVolumeId(null, false));
+        l.add(VolumeId.INVALID_VOLUME_ID);
       }
       blockVolumeIds.put(b, l);
     }
@@ -236,7 +236,7 @@ class BlockStorageLocationUtil {
         // Get the VolumeId by indexing into the list of VolumeIds
         // provided by the datanode
         byte[] volumeId = metaVolumeIds.get(volumeIndex);
-        HdfsVolumeId id = new HdfsVolumeId(volumeId, true);
+        HdfsVolumeId id = new HdfsVolumeId(volumeId);
         // Find out which index we are in the LocatedBlock's replicas
         LocatedBlock locBlock = extBlockToLocBlock.get(extBlock);
         DatanodeInfo[] dnInfos = locBlock.getLocations();

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

@@ -246,6 +246,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_HOSTS = "dfs.hosts";
   public static final String  DFS_HOSTS_EXCLUDE = "dfs.hosts.exclude";
   public static final String  DFS_CLIENT_LOCAL_INTERFACES = "dfs.client.local.interfaces";
+  public static final String  DFS_NAMENODE_AUDIT_LOGGERS_KEY = "dfs.namenode.audit.loggers";
+  public static final String  DFS_NAMENODE_DEFAULT_AUDIT_LOGGER_NAME = "default";
 
   // Much code in hdfs is not yet updated to use these keys.
   public static final String  DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY = "dfs.client.block.write.locateFollowingBlock.retries";

+ 42 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -1487,9 +1488,14 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
    */
   @Override
   public void hflush() throws IOException {
-    flushOrSync(false);
+    flushOrSync(false, EnumSet.noneOf(SyncFlag.class));
   }
 
+  @Override
+  public void hsync() throws IOException {
+    hsync(EnumSet.noneOf(SyncFlag.class));
+  }
+  
   /**
    * The expected semantics is all data have flushed out to all replicas 
    * and all replicas have done posix fsync equivalent - ie the OS has 
@@ -1498,17 +1504,35 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
    * Note that only the current block is flushed to the disk device.
    * To guarantee durable sync across block boundaries the stream should
    * be created with {@link CreateFlag#SYNC_BLOCK}.
+   * 
+   * @param syncFlags
+   *          Indicate the semantic of the sync. Currently used to specify
+   *          whether or not to update the block length in NameNode.
    */
-  @Override
-  public void hsync() throws IOException {
-    flushOrSync(true);
+  public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
+    flushOrSync(true, syncFlags);
   }
 
-  private void flushOrSync(boolean isSync) throws IOException {
+  /**
+   * Flush/Sync buffered data to DataNodes.
+   * 
+   * @param isSync
+   *          Whether or not to require all replicas to flush data to the disk
+   *          device
+   * @param syncFlags
+   *          Indicate extra detailed semantic of the flush/sync. Currently
+   *          mainly used to specify whether or not to update the file length in
+   *          the NameNode
+   * @throws IOException
+   */
+  private void flushOrSync(boolean isSync, EnumSet<SyncFlag> syncFlags)
+      throws IOException {
     dfsClient.checkOpen();
     isClosed();
     try {
       long toWaitFor;
+      long lastBlockLength = -1L;
+      boolean updateLength = syncFlags.contains(SyncFlag.UPDATE_LENGTH);
       synchronized (this) {
         /* Record current blockOffset. This might be changed inside
          * flushBuffer() where a partial checksum chunk might be flushed.
@@ -1572,13 +1596,20 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
       } // end synchronized
 
       waitForAckedSeqno(toWaitFor);
-
-      // If any new blocks were allocated since the last flush, 
-      // then persist block locations on namenode. 
-      //
-      if (persistBlocks.getAndSet(false)) {
+      
+      if (updateLength) {
+        synchronized (this) {
+          if (streamer != null && streamer.block != null) {
+            lastBlockLength = streamer.block.getNumBytes();
+          }
+        }
+      }
+      // If 1) any new blocks were allocated since the last flush, or 2) to
+      // update length in NN is requried, then persist block locations on
+      // namenode.
+      if (persistBlocks.getAndSet(false) || updateLength) {
         try {
-          dfsClient.namenode.fsync(src, dfsClient.clientName);
+          dfsClient.namenode.fsync(src, dfsClient.clientName, lastBlockLength);
         } catch (IOException ioe) {
           DFSClient.LOG.warn("Unable to persist blocks in hflush for " + src, ioe);
           // If we got an error here, it might be because some other thread called

+ 21 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.client;
 
 import java.io.IOException;
+import java.util.EnumSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -56,4 +57,24 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
   public synchronized int getCurrentBlockReplication() throws IOException {
     return ((DFSOutputStream)getWrappedStream()).getCurrentBlockReplication();
   }
+  
+  /**
+   * Sync buffered data to DataNodes (flush to disk devices).
+   * 
+   * @param syncFlags
+   *          Indicate the detailed semantic and actions of the hsync.
+   * @throws IOException
+   * @see FSDataOutputStream#hsync()
+   */
+  public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
+    ((DFSOutputStream) getWrappedStream()).hsync(syncFlags);
+  }
+  
+  public static enum SyncFlag {
+    /**
+     * When doing sync to DataNodes, also update the metadata (block
+     * length) in the NameNode
+     */
+    UPDATE_LENGTH;
+  }
 }

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

@@ -815,14 +815,15 @@ public interface ClientProtocol {
    * The file must be currently open for writing.
    * @param src The string representation of the path
    * @param client The string representation of the client
-   * 
+   * @param lastBlockLength The length of the last block (under construction) 
+   *                        to be reported to NameNode 
    * @throws AccessControlException permission denied
    * @throws FileNotFoundException file <code>src</code> is not found
    * @throws UnresolvedLinkException if <code>src</code> contains a symlink. 
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public void fsync(String src, String client) 
+  public void fsync(String src, String client, long lastBlockLength) 
       throws AccessControlException, FileNotFoundException, 
       UnresolvedLinkException, IOException;
 

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

@@ -688,7 +688,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public FsyncResponseProto fsync(RpcController controller,
       FsyncRequestProto req) throws ServiceException {
     try {
-      server.fsync(req.getSrc(), req.getClient());
+      server.fsync(req.getSrc(), req.getClient(), req.getLastBlockLength());
       return VOID_FSYNC_RESPONSE;
     } catch (IOException e) {
       throw new ServiceException(e);

+ 5 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -659,12 +659,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public void fsync(String src, String client) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
-    FsyncRequestProto req = FsyncRequestProto.newBuilder()
-        .setSrc(src)
-        .setClient(client)
-        .build();
+  public void fsync(String src, String client, long lastBlockLength)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    FsyncRequestProto req = FsyncRequestProto.newBuilder().setSrc(src)
+        .setClient(client).setLastBlockLength(lastBlockLength).build();
     try {
       rpcProxy.fsync(null, req);
     } catch (ServiceException e) {

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
@@ -1232,9 +1233,9 @@ public class PBHelper {
     if (s == null) return null;
     switch (s.getState()) {
     case ACTIVE:
-      return new NNHAStatusHeartbeat(NNHAStatusHeartbeat.State.ACTIVE, s.getTxid());
+      return new NNHAStatusHeartbeat(HAServiceState.ACTIVE, s.getTxid());
     case STANDBY:
-      return new NNHAStatusHeartbeat(NNHAStatusHeartbeat.State.STANDBY, s.getTxid());
+      return new NNHAStatusHeartbeat(HAServiceState.STANDBY, s.getTxid());
     default:
       throw new IllegalArgumentException("Unexpected NNHAStatusHeartbeat.State:" + s.getState());
     }

+ 168 - 157
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -75,6 +75,7 @@ import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocat
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
@@ -168,7 +169,7 @@ import org.apache.hadoop.util.ToolRunner;
  * <ol>
  * <li>The cluster is balanced. Exiting
  * <li>No block can be moved. Exiting...
- * <li>No block has been moved for 3 iterations. Exiting...
+ * <li>No block has been moved for 5 iterations. Exiting...
  * <li>Received an IO exception: failure reason. Exiting...
  * <li>Another balancer is running. Exiting...
  * </ol>
@@ -222,7 +223,7 @@ public class Balancer {
   private Map<String, BalancerDatanode> datanodes
                  = new HashMap<String, BalancerDatanode>();
   
-  private NetworkTopology cluster = new NetworkTopology();
+  private NetworkTopology cluster;
   
   final static private int MOVER_THREAD_POOL_SIZE = 1000;
   final private ExecutorService moverExecutor = 
@@ -249,7 +250,7 @@ public class Balancer {
      * Return true if a block and its proxy are chosen; false otherwise
      */
     private boolean chooseBlockAndProxy() {
-      // iterate all source's blocks until find a good one    
+      // iterate all source's blocks until find a good one
       for (Iterator<BalancerBlock> blocks=
         source.getBlockIterator(); blocks.hasNext();) {
         if (markMovedIfGoodBlock(blocks.next())) {
@@ -293,22 +294,35 @@ public class Balancer {
      * @return true if a proxy is found; otherwise false
      */
     private boolean chooseProxySource() {
-      // check if there is replica which is on the same rack with the target
+      final DatanodeInfo targetDN = target.getDatanode();
+      boolean find = false;
       for (BalancerDatanode loc : block.getLocations()) {
-        if (cluster.isOnSameRack(loc.getDatanode(), target.getDatanode())) {
-          if (loc.addPendingBlock(this)) {
-            proxySource = loc;
+        // check if there is replica which is on the same rack with the target
+        if (cluster.isOnSameRack(loc.getDatanode(), targetDN) && addTo(loc)) {
+          find = true;
+          // if cluster is not nodegroup aware or the proxy is on the same 
+          // nodegroup with target, then we already find the nearest proxy
+          if (!cluster.isNodeGroupAware() 
+              || cluster.isOnSameNodeGroup(loc.getDatanode(), targetDN)) {
             return true;
           }
         }
-      }
-      // find out a non-busy replica
-      for (BalancerDatanode loc : block.getLocations()) {
-        if (loc.addPendingBlock(this)) {
-          proxySource = loc;
-          return true;
+        
+        if (!find) {
+          // find out a non-busy replica out of rack of target
+          find = addTo(loc);
         }
       }
+      
+      return find;
+    }
+    
+    // add a BalancerDatanode as proxy source for specific block movement
+    private boolean addTo(BalancerDatanode bdn) {
+      if (bdn.addPendingBlock(this)) {
+        proxySource = bdn;
+        return true;
+      }
       return false;
     }
     
@@ -544,7 +558,7 @@ public class Balancer {
     }
     
     /** Decide if still need to move more bytes */
-    protected boolean isMoveQuotaFull() {
+    protected boolean hasSpaceForScheduling() {
       return scheduledSize<maxSize2Move;
     }
 
@@ -686,7 +700,7 @@ public class Balancer {
         NodeTask task = tasks.next();
         BalancerDatanode target = task.getDatanode();
         PendingBlockMove pendingBlock = new PendingBlockMove();
-        if ( target.addPendingBlock(pendingBlock) ) { 
+        if (target.addPendingBlock(pendingBlock)) { 
           // target is not busy, so do a tentative block allocation
           pendingBlock.source = this;
           pendingBlock.target = target;
@@ -787,9 +801,10 @@ public class Balancer {
    */
   private static void checkReplicationPolicyCompatibility(Configuration conf
       ) throws UnsupportedActionException {
-    if (BlockPlacementPolicy.getInstance(conf, null, null).getClass() != 
-        BlockPlacementPolicyDefault.class) {
-      throw new UnsupportedActionException("Balancer without BlockPlacementPolicyDefault");
+    if (BlockPlacementPolicy.getInstance(conf, null, null) instanceof 
+        BlockPlacementPolicyDefault) {
+      throw new UnsupportedActionException(
+          "Balancer without BlockPlacementPolicyDefault");
     }
   }
 
@@ -804,6 +819,7 @@ public class Balancer {
     this.threshold = p.threshold;
     this.policy = p.policy;
     this.nnc = theblockpool;
+    cluster = NetworkTopology.getInstance(conf);
   }
   
   /* Shuffle datanode array */
@@ -907,17 +923,53 @@ public class Balancer {
     LOG.info(nodes.size() + " " + name + ": " + nodes);
   }
 
-  /* Decide all <source, target> pairs and
+  /** A matcher interface for matching nodes. */
+  private interface Matcher {
+    /** Given the cluster topology, does the left node match the right node? */
+    boolean match(NetworkTopology cluster, Node left,  Node right);
+  }
+
+  /** Match datanodes in the same node group. */
+  static final Matcher SAME_NODE_GROUP = new Matcher() {
+    @Override
+    public boolean match(NetworkTopology cluster, Node left, Node right) {
+      return cluster.isOnSameNodeGroup(left, right);
+    }
+  };
+
+  /** Match datanodes in the same rack. */
+  static final Matcher SAME_RACK = new Matcher() {
+    @Override
+    public boolean match(NetworkTopology cluster, Node left, Node right) {
+      return cluster.isOnSameRack(left, right);
+    }
+  };
+
+  /** Match any datanode with any other datanode. */
+  static final Matcher ANY_OTHER = new Matcher() {
+    @Override
+    public boolean match(NetworkTopology cluster, Node left, Node right) {
+      return left != right;
+    }
+  };
+
+  /**
+   * Decide all <source, target> pairs and
    * the number of bytes to move from a source to a target
    * Maximum bytes to be moved per node is
    * Min(1 Band worth of bytes,  MAX_SIZE_TO_MOVE).
    * Return total number of bytes to move in this iteration
    */
   private long chooseNodes() {
-    // Match nodes on the same rack first
-    chooseNodes(true);
-    // Then match nodes on different racks
-    chooseNodes(false);
+    // First, match nodes on the same node group if cluster is node group aware
+    if (cluster.isNodeGroupAware()) {
+      chooseNodes(SAME_NODE_GROUP);
+    }
+    
+    // Then, match nodes on the same rack
+    chooseNodes(SAME_RACK);
+    // At last, match all remaining nodes
+    chooseNodes(ANY_OTHER);
     
     assert (datanodes.size() >= sources.size()+targets.size())
       : "Mismatched number of datanodes (" +
@@ -932,162 +984,94 @@ public class Balancer {
     return bytesToMove;
   }
 
-  /* if onRack is true, decide all <source, target> pairs
-   * where source and target are on the same rack; Otherwise
-   * decide all <source, target> pairs where source and target are
-   * on different racks
-   */
-  private void chooseNodes(boolean onRack) {
+  /** Decide all <source, target> pairs according to the matcher. */
+  private void chooseNodes(final Matcher matcher) {
     /* first step: match each overUtilized datanode (source) to
      * one or more underUtilized datanodes (targets).
      */
-    chooseTargets(underUtilizedDatanodes.iterator(), onRack);
+    chooseDatanodes(overUtilizedDatanodes, underUtilizedDatanodes, matcher);
     
     /* match each remaining overutilized datanode (source) to 
      * below average utilized datanodes (targets).
      * Note only overutilized datanodes that haven't had that max bytes to move
      * satisfied in step 1 are selected
      */
-    chooseTargets(belowAvgUtilizedDatanodes.iterator(), onRack);
+    chooseDatanodes(overUtilizedDatanodes, belowAvgUtilizedDatanodes, matcher);
 
-    /* match each remaining underutilized datanode to 
-     * above average utilized datanodes.
+    /* match each remaining underutilized datanode (target) to 
+     * above average utilized datanodes (source).
      * Note only underutilized datanodes that have not had that max bytes to
      * move satisfied in step 1 are selected.
      */
-    chooseSources(aboveAvgUtilizedDatanodes.iterator(), onRack);
+    chooseDatanodes(underUtilizedDatanodes, aboveAvgUtilizedDatanodes, matcher);
   }
-   
-  /* choose targets from the target candidate list for each over utilized
-   * source datanode. OnRackTarget determines if the chosen target 
-   * should be on the same rack as the source
+
+  /**
+   * For each datanode, choose matching nodes from the candidates. Either the
+   * datanodes or the candidates are source nodes with (utilization > Avg), and
+   * the others are target nodes with (utilization < Avg).
    */
-  private void chooseTargets(  
-      Iterator<BalancerDatanode> targetCandidates, boolean onRackTarget ) {
-    for (Iterator<Source> srcIterator = overUtilizedDatanodes.iterator();
-        srcIterator.hasNext();) {
-      Source source = srcIterator.next();
-      while (chooseTarget(source, targetCandidates, onRackTarget)) {
-      }
-      if (!source.isMoveQuotaFull()) {
-        srcIterator.remove();
+  private <D extends BalancerDatanode, C extends BalancerDatanode> void 
+      chooseDatanodes(Collection<D> datanodes, Collection<C> candidates,
+          Matcher matcher) {
+    for (Iterator<D> i = datanodes.iterator(); i.hasNext();) {
+      final D datanode = i.next();
+      for(; chooseForOneDatanode(datanode, candidates, matcher); );
+      if (!datanode.hasSpaceForScheduling()) {
+        i.remove();
       }
     }
-    return;
-  }
-  
-  /* choose sources from the source candidate list for each under utilized
-   * target datanode. onRackSource determines if the chosen source 
-   * should be on the same rack as the target
-   */
-  private void chooseSources(
-      Iterator<Source> sourceCandidates, boolean onRackSource) {
-    for (Iterator<BalancerDatanode> targetIterator = 
-      underUtilizedDatanodes.iterator(); targetIterator.hasNext();) {
-      BalancerDatanode target = targetIterator.next();
-      while (chooseSource(target, sourceCandidates, onRackSource)) {
-      }
-      if (!target.isMoveQuotaFull()) {
-        targetIterator.remove();
-      }
-    }
-    return;
   }
 
-  /* For the given source, choose targets from the target candidate list.
-   * OnRackTarget determines if the chosen target 
-   * should be on the same rack as the source
+  /**
+   * For the given datanode, choose a candidate and then schedule it.
+   * @return true if a candidate is chosen; false if no candidates is chosen.
    */
-  private boolean chooseTarget(Source source,
-      Iterator<BalancerDatanode> targetCandidates, boolean onRackTarget) {
-    if (!source.isMoveQuotaFull()) {
+  private <C extends BalancerDatanode> boolean chooseForOneDatanode(
+      BalancerDatanode dn, Collection<C> candidates, Matcher matcher) {
+    final Iterator<C> i = candidates.iterator();
+    final C chosen = chooseCandidate(dn, i, matcher);
+
+    if (chosen == null) {
       return false;
     }
-    boolean foundTarget = false;
-    BalancerDatanode target = null;
-    while (!foundTarget && targetCandidates.hasNext()) {
-      target = targetCandidates.next();
-      if (!target.isMoveQuotaFull()) {
-        targetCandidates.remove();
-        continue;
-      }
-      if (onRackTarget) {
-        // choose from on-rack nodes
-        if (cluster.isOnSameRack(source.datanode, target.datanode)) {
-          foundTarget = true;
-        }
-      } else {
-        // choose from off-rack nodes
-        if (!cluster.isOnSameRack(source.datanode, target.datanode)) {
-          foundTarget = true;
-        }
-      }
+    if (dn instanceof Source) {
+      matchSourceWithTargetToMove((Source)dn, chosen);
+    } else {
+      matchSourceWithTargetToMove((Source)chosen, dn);
     }
-    if (foundTarget) {
-      assert(target != null):"Choose a null target";
-      long size = Math.min(source.availableSizeToMove(),
-          target.availableSizeToMove());
-      NodeTask nodeTask = new NodeTask(target, size);
-      source.addNodeTask(nodeTask);
-      target.incScheduledSize(nodeTask.getSize());
-      sources.add(source);
-      targets.add(target);
-      if (!target.isMoveQuotaFull()) {
-        targetCandidates.remove();
-      }
-      LOG.info("Decided to move "+StringUtils.byteDesc(size)+" bytes from "
-          +source.datanode + " to " + target.datanode);
-      return true;
+    if (!chosen.hasSpaceForScheduling()) {
+      i.remove();
     }
-    return false;
+    return true;
   }
   
-  /* For the given target, choose sources from the source candidate list.
-   * OnRackSource determines if the chosen source 
-   * should be on the same rack as the target
-   */
-  private boolean chooseSource(BalancerDatanode target,
-      Iterator<Source> sourceCandidates, boolean onRackSource) {
-    if (!target.isMoveQuotaFull()) {
-      return false;
-    }
-    boolean foundSource = false;
-    Source source = null;
-    while (!foundSource && sourceCandidates.hasNext()) {
-      source = sourceCandidates.next();
-      if (!source.isMoveQuotaFull()) {
-        sourceCandidates.remove();
-        continue;
-      }
-      if (onRackSource) {
-        // choose from on-rack nodes
-        if ( cluster.isOnSameRack(source.getDatanode(), target.getDatanode())) {
-          foundSource = true;
-        }
-      } else {
-        // choose from off-rack nodes
-        if (!cluster.isOnSameRack(source.datanode, target.datanode)) {
-          foundSource = true;
+  private void matchSourceWithTargetToMove(
+      Source source, BalancerDatanode target) {
+    long size = Math.min(source.availableSizeToMove(), target.availableSizeToMove());
+    NodeTask nodeTask = new NodeTask(target, size);
+    source.addNodeTask(nodeTask);
+    target.incScheduledSize(nodeTask.getSize());
+    sources.add(source);
+    targets.add(target);
+    LOG.info("Decided to move "+StringUtils.byteDesc(size)+" bytes from "
+        +source.datanode.getName() + " to " + target.datanode.getName());
+  }
+  
+  /** Choose a candidate for the given datanode. */
+  private <D extends BalancerDatanode, C extends BalancerDatanode>
+      C chooseCandidate(D dn, Iterator<C> candidates, Matcher matcher) {
+    if (dn.hasSpaceForScheduling()) {
+      for(; candidates.hasNext(); ) {
+        final C c = candidates.next();
+        if (!c.hasSpaceForScheduling()) {
+          candidates.remove();
+        } else if (matcher.match(cluster, dn.getDatanode(), c.getDatanode())) {
+          return c;
         }
       }
     }
-    if (foundSource) {
-      assert(source != null):"Choose a null source";
-      long size = Math.min(source.availableSizeToMove(),
-          target.availableSizeToMove());
-      NodeTask nodeTask = new NodeTask(target, size);
-      source.addNodeTask(nodeTask);
-      target.incScheduledSize(nodeTask.getSize());
-      sources.add(source);
-      targets.add(target);
-      if ( !source.isMoveQuotaFull()) {
-        sourceCandidates.remove();
-      }
-      LOG.info("Decided to move "+StringUtils.byteDesc(size)+" bytes from "
-          +source.datanode + " to " + target.datanode);
-      return true;
-    }
-    return false;
+    return null;
   }
 
   private static class BytesMoved {
@@ -1226,6 +1210,10 @@ public class Balancer {
     if (block.isLocatedOnDatanode(target)) {
       return false;
     }
+    if (cluster.isNodeGroupAware() && 
+        isOnSameNodeGroupWithReplicas(target, block, source)) {
+      return false;
+    }
 
     boolean goodBlock = false;
     if (cluster.isOnSameRack(source.getDatanode(), target.getDatanode())) {
@@ -1257,10 +1245,32 @@ public class Balancer {
     }
     return goodBlock;
   }
-  
+
+  /**
+   * Check if there are any replica (other than source) on the same node group
+   * with target. If true, then target is not a good candidate for placing 
+   * specific block replica as we don't want 2 replicas under the same nodegroup 
+   * after balance.
+   * @param target targetDataNode
+   * @param block dataBlock
+   * @param source sourceDataNode
+   * @return true if there are any replica (other than source) on the same node
+   * group with target
+   */
+  private boolean isOnSameNodeGroupWithReplicas(BalancerDatanode target,
+      BalancerBlock block, Source source) {
+    for (BalancerDatanode loc : block.locations) {
+      if (loc != source && 
+        cluster.isOnSameNodeGroup(loc.getDatanode(), target.getDatanode())) {
+          return true;
+        }
+      }
+    return false;
+  }
+
   /* reset all fields in a balancer preparing for the next iteration */
-  private void resetData() {
-    this.cluster = new NetworkTopology();
+  private void resetData(Configuration conf) {
+    this.cluster = NetworkTopology.getInstance(conf);
     this.overUtilizedDatanodes.clear();
     this.aboveAvgUtilizedDatanodes.clear();
     this.belowAvgUtilizedDatanodes.clear();
@@ -1331,7 +1341,8 @@ public class Balancer {
   }
 
   /** Run an iteration for all datanodes. */
-  private ReturnStatus run(int iteration, Formatter formatter) {
+  private ReturnStatus run(int iteration, Formatter formatter,
+      Configuration conf) {
     try {
       /* get all live datanodes of a cluster and their disk usage
        * decide the number of bytes need to be moved
@@ -1385,7 +1396,7 @@ public class Balancer {
       }
 
       // clean all lists
-      resetData();
+      resetData(conf);
       return ReturnStatus.IN_PROGRESS;
     } catch (IllegalArgumentException e) {
       System.out.println(e + ".  Exiting ...");
@@ -1433,7 +1444,7 @@ public class Balancer {
         Collections.shuffle(connectors);
         for(NameNodeConnector nnc : connectors) {
           final Balancer b = new Balancer(nnc, p, conf);
-          final ReturnStatus r = b.run(iteration, formatter);
+          final ReturnStatus r = b.run(iteration, formatter, conf);
           if (r == ReturnStatus.IN_PROGRESS) {
             done = false;
           } else if (r != ReturnStatus.SUCCESS) {
@@ -1527,7 +1538,7 @@ public class Balancer {
       if (args != null) {
         try {
           for(int i = 0; i < args.length; i++) {
-            checkArgument(args.length >= 2, "args = " + Arrays.toString(args));           
+            checkArgument(args.length >= 2, "args = " + Arrays.toString(args));
             if ("-threshold".equalsIgnoreCase(args[i])) {
               i++;
               try {

+ 24 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -152,8 +152,9 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       
     List<DatanodeDescriptor> results = 
       new ArrayList<DatanodeDescriptor>(chosenNodes);
-    for (Node node:chosenNodes) {
-      excludedNodes.put(node, node);
+    for (DatanodeDescriptor node:chosenNodes) {
+      // add localMachine and related nodes to excludedNodes
+      addToExcludedNodes(node, excludedNodes);
       adjustExcludedNodes(excludedNodes, node);
     }
       
@@ -235,7 +236,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                + totalReplicasExpected + "\n"
                + e.getMessage());
       if (avoidStaleNodes) {
-        // ecxludedNodes now has - initial excludedNodes, any nodes that were
+        // excludedNodes now has - initial excludedNodes, any nodes that were
         // chosen and nodes that were tried but were not chosen because they
         // were stale, decommissioned or for any other reason a node is not
         // chosen for write. Retry again now not avoiding stale node
@@ -273,6 +274,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         if (isGoodTarget(localMachine, blocksize, maxNodesPerRack, false,
             results, avoidStaleNodes)) {
           results.add(localMachine);
+          // add localMachine and related nodes to excludedNode
+          addToExcludedNodes(localMachine, excludedNodes);
           return localMachine;
         }
       } 
@@ -281,7 +284,19 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     return chooseLocalRack(localMachine, excludedNodes, blocksize,
         maxNodesPerRack, results, avoidStaleNodes);
   }
-    
+  
+  /**
+   * Add <i>localMachine</i> and related nodes to <i>excludedNodes</i>
+   * for next replica choosing. In sub class, we can add more nodes within
+   * the same failure domain of localMachine
+   * @return number of new excluded nodes
+   */
+  protected int addToExcludedNodes(DatanodeDescriptor localMachine,
+      HashMap<Node, Node> excludedNodes) {
+    Node node = excludedNodes.put(localMachine, localMachine);
+    return node == null?1:0;
+  }
+
   /* choose one node from the rack that <i>localMachine</i> is on.
    * if no such node is available, choose one node from the rack where
    * a second replica is on.
@@ -392,6 +407,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         if (isGoodTarget(chosenNode, blocksize, 
                 maxNodesPerRack, results, avoidStaleNodes)) {
           results.add(chosenNode);
+          // add chosenNode and related nodes to excludedNode
+          addToExcludedNodes(chosenNode, excludedNodes);
           adjustExcludedNodes(excludedNodes, chosenNode);
           return chosenNode;
         } else {
@@ -441,6 +458,9 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
               maxNodesPerRack, results, avoidStaleNodes)) {
           numOfReplicas--;
           results.add(chosenNode);
+          // add chosenNode and related nodes to excludedNode
+          int newExcludedNodes = addToExcludedNodes(chosenNode, excludedNodes);
+          numOfAvailableNodes -= newExcludedNodes;
           adjustExcludedNodes(excludedNodes, chosenNode);
         } else {
           badTarget = true;

+ 21 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java

@@ -240,6 +240,27 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
     String nodeGroupString = cur.getNetworkLocation();
     return NetworkTopology.getFirstHalf(nodeGroupString);
   }
+  
+  /**
+   * Find other nodes in the same nodegroup of <i>localMachine</i> and add them
+   * into <i>excludeNodes</i> as replica should not be duplicated for nodes 
+   * within the same nodegroup
+   * @return number of new excluded nodes
+   */
+  protected int addToExcludedNodes(DatanodeDescriptor localMachine,
+      HashMap<Node, Node> excludedNodes) {
+    int countOfExcludedNodes = 0;
+    String nodeGroupScope = localMachine.getNetworkLocation();
+    List<Node> leafNodes = clusterMap.getLeaves(nodeGroupScope);
+    for (Node leafNode : leafNodes) {
+      Node node = excludedNodes.put(leafNode, leafNode);
+      if (node == null) {
+        // not a existing node in excludedNodes
+        countOfExcludedNodes++;
+      }
+    }
+    return countOfExcludedNodes;
+  }
 
   /**
    * Pick up replica node set for deleting replica as over-replicated. 

+ 1 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -39,7 +39,6 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -151,10 +150,7 @@ public class DatanodeManager {
     this.namesystem = namesystem;
     this.blockManager = blockManager;
     
-    Class<? extends NetworkTopology> networkTopologyClass =
-        conf.getClass(CommonConfigurationKeysPublic.NET_TOPOLOGY_IMPL_KEY,
-            NetworkTopology.class, NetworkTopology.class);
-    networktopology = ReflectionUtils.newInstance(networkTopologyClass, conf);
+    networktopology = NetworkTopology.getInstance(conf);
 
     this.heartbeatManager = new HeartbeatManager(namesystem, blockManager, conf);
 

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

@@ -26,6 +26,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -411,7 +412,7 @@ class BPOfferService {
     final long txid = nnHaState.getTxId();
     
     final boolean nnClaimsActive =
-      nnHaState.getState() == NNHAStatusHeartbeat.State.ACTIVE;
+      nnHaState.getState() == HAServiceState.ACTIVE;
     final boolean bposThinksActive = bpServiceToActive == actor;
     final boolean isMoreRecentClaim = txid > lastActiveClaimTxId; 
     

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

@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.net.InetAddress;
+import java.security.Principal;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+
+/**
+ * Interface defining an audit logger.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface AuditLogger {
+
+  /**
+   * Called during initialization of the logger.
+   *
+   * @param conf The configuration object.
+   */
+  void initialize(Configuration conf);
+
+  /**
+   * Called to log an audit event.
+   * <p>
+   * This method must return as quickly as possible, since it's called
+   * in a critical section of the NameNode's operation.
+   *
+   * @param succeeded Whether authorization succeeded.
+   * @param userName Name of the user executing the request.
+   * @param addr Remote address of the request.
+   * @param cmd The requested command.
+   * @param src Path of affected source file.
+   * @param dst Path of affected destination file (if any).
+   * @param stat File information for operations that change the file's
+   *             metadata (permissions, owner, times, etc).
+   */
+  void logAuditEvent(boolean succeeded, String userName,
+      InetAddress addr, String cmd, String src, String dst,
+      FileStatus stat);
+
+}

+ 51 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java

@@ -24,6 +24,7 @@ import java.net.SocketTimeoutException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.ha.ServiceFailedException;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.NameNodeProxies;
@@ -35,6 +36,7 @@ import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
 import org.apache.hadoop.hdfs.server.protocol.FenceResponse;
 import org.apache.hadoop.hdfs.server.protocol.JournalInfo;
 import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
@@ -414,14 +416,23 @@ public class BackupNode extends NameNode {
       + HdfsConstants.LAYOUT_VERSION + " actual "+ nsInfo.getLayoutVersion();
     return nsInfo;
   }
-  
+
   @Override
+  protected String getNameServiceId(Configuration conf) {
+    return DFSUtil.getBackupNameServiceId(conf);
+  }
+
+  protected HAState createHAState() {
+    return new BackupState();
+  }
+
+  @Override // NameNode
   protected NameNodeHAContext createHAContext() {
     return new BNHAContext();
   }
-  
+
   private class BNHAContext extends NameNodeHAContext {
-    @Override // NameNode
+    @Override // NameNodeHAContext
     public void checkOperation(OperationCategory op)
         throws StandbyException {
       if (op == OperationCategory.UNCHECKED ||
@@ -435,10 +446,42 @@ public class BackupNode extends NameNode {
         throw new StandbyException(msg);
       }
     }
-  }
-  
-  @Override
-  protected String getNameServiceId(Configuration conf) {
-    return DFSUtil.getBackupNameServiceId(conf);
+
+    @Override // NameNodeHAContext
+    public void prepareToStopStandbyServices() throws ServiceFailedException {
+    }
+
+    /**
+     * Start services for BackupNode.
+     * <p>
+     * The following services should be muted
+     * (not run or not pass any control commands to DataNodes)
+     * on BackupNode:
+     * {@link LeaseManager.Monitor} protected by SafeMode.
+     * {@link BlockManager.ReplicationMonitor} protected by SafeMode.
+     * {@link HeartbeatManager.Monitor} protected by SafeMode.
+     * {@link DecommissionManager.Monitor} need to prohibit refreshNodes().
+     * {@link PendingReplicationBlocks.PendingReplicationMonitor} harmless,
+     * because ReplicationMonitor is muted.
+     */
+    @Override
+    public void startActiveServices() throws IOException {
+      try {
+        namesystem.startActiveServices();
+      } catch (Throwable t) {
+        doImmediateShutdown(t);
+      }
+    }
+
+    @Override
+    public void stopActiveServices() throws IOException {
+      try {
+        if (namesystem != null) {
+          namesystem.stopActiveServices();
+        }
+      } catch (Throwable t) {
+        doImmediateShutdown(t);
+      }
+    }
   }
 }

+ 53 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupState.java

@@ -0,0 +1,53 @@
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.ha.ServiceFailedException;
+import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
+import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
+import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
+import org.apache.hadoop.ipc.StandbyException;
+
+@InterfaceAudience.Private
+public class BackupState extends HAState {
+
+  public BackupState() {
+    super(HAServiceState.STANDBY);
+  }
+
+  @Override // HAState
+  public void checkOperation(HAContext context, OperationCategory op)
+      throws StandbyException {
+    context.checkOperation(op);
+  }
+
+  @Override // HAState
+  public boolean shouldPopulateReplQueues() {
+    return false;
+  }
+
+  @Override // HAState
+  public void enterState(HAContext context) throws ServiceFailedException {
+    try {
+      context.startActiveServices();
+    } catch (IOException e) {
+      throw new ServiceFailedException("Failed to start backup services", e);
+    }
+  }
+
+  @Override // HAState
+  public void exitState(HAContext context) throws ServiceFailedException {
+    try {
+      context.stopActiveServices();
+    } catch (IOException e) {
+      throw new ServiceFailedException("Failed to stop backup services", e);
+    }
+  }
+
+  @Override // HAState
+  public void prepareToExitState(HAContext context) throws ServiceFailedException {
+    context.prepareToStopStandbyServices();
+  }
+}

+ 33 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -575,6 +575,8 @@ public class FSDirectory implements Closeable {
         // update modification time of dst and the parent of src
         srcInodes[srcInodes.length-2].setModificationTime(timestamp);
         dstInodes[dstInodes.length-2].setModificationTime(timestamp);
+        // update moved leases with new filename
+        getFSNamesystem().unprotectedChangeLease(src, dst);        
         return true;
       }
     } finally {
@@ -729,6 +731,8 @@ public class FSDirectory implements Closeable {
         }
         srcInodes[srcInodes.length - 2].setModificationTime(timestamp);
         dstInodes[dstInodes.length - 2].setModificationTime(timestamp);
+        // update moved lease with new filename
+        getFSNamesystem().unprotectedChangeLease(src, dst);
 
         // Collect the blocks and remove the lease for previous dst
         int filesDeleted = 0;
@@ -1071,31 +1075,39 @@ public class FSDirectory implements Closeable {
       throws IOException, UnresolvedLinkException {    
     writeLock();
     try {
-      //
-      // Remove the node from the namespace 
-      //
-      if (!oldnode.removeNode()) {
-        NameNode.stateChangeLog.warn("DIR* FSDirectory.replaceNode: " +
-                                     "failed to remove " + path);
-        throw new IOException("FSDirectory.replaceNode: " +
-                              "failed to remove " + path);
-      } 
-      
-      /* Currently oldnode and newnode are assumed to contain the same
-       * blocks. Otherwise, blocks need to be removed from the blocksMap.
-       */
-      rootDir.addINode(path, newnode); 
-
-      int index = 0;
-      for (BlockInfo b : newnode.getBlocks()) {
-        BlockInfo info = getBlockManager().addBlockCollection(b, newnode);
-        newnode.setBlock(index, info); // inode refers to the block in BlocksMap
-        index++;
-      }
+      unprotectedReplaceNode(path, oldnode, newnode);
     } finally {
       writeUnlock();
     }
   }
+  
+  void unprotectedReplaceNode(String path, INodeFile oldnode, INodeFile newnode)
+      throws IOException, UnresolvedLinkException {
+    assert hasWriteLock();
+    INodeDirectory parent = oldnode.parent;
+    // Remove the node from the namespace 
+    if (!oldnode.removeNode()) {
+      NameNode.stateChangeLog.warn("DIR* FSDirectory.replaceNode: " +
+                                   "failed to remove " + path);
+      throw new IOException("FSDirectory.replaceNode: " +
+                            "failed to remove " + path);
+    } 
+    
+    // Parent should be non-null, otherwise oldnode.removeNode() will return
+    // false
+    newnode.setLocalName(oldnode.getLocalNameBytes());
+    parent.addChild(newnode, true);
+    
+    /* Currently oldnode and newnode are assumed to contain the same
+     * blocks. Otherwise, blocks need to be removed from the blocksMap.
+     */
+    int index = 0;
+    for (BlockInfo b : newnode.getBlocks()) {
+      BlockInfo info = getBlockManager().addBlockCollection(b, newnode);
+      newnode.setBlock(index, info); // inode refers to the block in BlocksMap
+      index++;
+    }
+  }
 
   /**
    * Get a partial listing of the indicated directory

+ 14 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -878,6 +878,11 @@ public class FSEditLog implements LogsPurgeable {
     return journalSet;
   }
   
+  @VisibleForTesting
+  synchronized void setJournalSetForTesting(JournalSet js) {
+    this.journalSet = js;
+  }
+  
   /**
    * Used only by tests.
    */
@@ -1031,9 +1036,18 @@ public class FSEditLog implements LogsPurgeable {
 
   /**
    * Archive any log files that are older than the given txid.
+   * 
+   * If the edit log is not open for write, then this call returns with no
+   * effect.
    */
   @Override
   public synchronized void purgeLogsOlderThan(final long minTxIdToKeep) {
+    // Should not purge logs unless they are open for write.
+    // This prevents the SBN from purging logs on shared storage, for example.
+    if (!isOpenForWrite()) {
+      return;
+    }
+    
     assert curSegmentTxId == HdfsConstants.INVALID_TXID || // on format this is no-op
       minTxIdToKeep <= curSegmentTxId :
       "cannot purge logs older than txid " + minTxIdToKeep +

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

@@ -31,7 +31,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
@@ -322,7 +321,7 @@ public class FSEditLogLoader {
         INodeFileUnderConstruction ucFile = (INodeFileUnderConstruction) oldFile;
         fsNamesys.leaseManager.removeLeaseWithPrefixPath(addCloseOp.path);
         INodeFile newFile = ucFile.convertToInodeFile();
-        fsDir.replaceNode(addCloseOp.path, ucFile, newFile);
+        fsDir.unprotectedReplaceNode(addCloseOp.path, ucFile, newFile);
       }
       break;
     }
@@ -360,10 +359,8 @@ public class FSEditLogLoader {
     }
     case OP_RENAME_OLD: {
       RenameOldOp renameOp = (RenameOldOp)op;
-      HdfsFileStatus dinfo = fsDir.getFileInfo(renameOp.dst, false);
       fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
                                 renameOp.timestamp);
-      fsNamesys.unprotectedChangeLease(renameOp.src, renameOp.dst, dinfo);
       break;
     }
     case OP_DELETE: {
@@ -433,11 +430,8 @@ public class FSEditLogLoader {
     }
     case OP_RENAME: {
       RenameOp renameOp = (RenameOp)op;
-
-      HdfsFileStatus dinfo = fsDir.getFileInfo(renameOp.dst, false);
       fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
                                 renameOp.timestamp, renameOp.options);
-      fsNamesys.unprotectedChangeLease(renameOp.src, renameOp.dst, dinfo);
       break;
     }
     case OP_GET_DELEGATION_TOKEN: {

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

@@ -34,6 +34,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DEFAULT_AUDIT_LOGGER_NAME;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_DEFAULT;
@@ -111,6 +113,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.Options;
@@ -121,6 +124,7 @@ import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.ServiceFailedException;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
@@ -163,12 +167,10 @@ import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.INodesInPath;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
-import org.apache.hadoop.hdfs.server.namenode.ha.ActiveState;
 import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
 import org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer;
-import org.apache.hadoop.hdfs.server.namenode.ha.StandbyState;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
@@ -246,32 +248,32 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       }
   };
 
-  private static final void logAuditEvent(UserGroupInformation ugi,
+  private boolean isAuditEnabled() {
+    return !isDefaultAuditLogger || auditLog.isInfoEnabled();
+  }
+
+  private void logAuditEvent(UserGroupInformation ugi,
       InetAddress addr, String cmd, String src, String dst,
       HdfsFileStatus stat) {
     logAuditEvent(true, ugi, addr, cmd, src, dst, stat);
   }
 
-  private static final void logAuditEvent(boolean succeeded,
+  private void logAuditEvent(boolean succeeded,
       UserGroupInformation ugi, InetAddress addr, String cmd, String src,
       String dst, HdfsFileStatus stat) {
-    final StringBuilder sb = auditBuffer.get();
-    sb.setLength(0);
-    sb.append("allowed=").append(succeeded).append("\t");
-    sb.append("ugi=").append(ugi).append("\t");
-    sb.append("ip=").append(addr).append("\t");
-    sb.append("cmd=").append(cmd).append("\t");
-    sb.append("src=").append(src).append("\t");
-    sb.append("dst=").append(dst).append("\t");
-    if (null == stat) {
-      sb.append("perm=null");
-    } else {
-      sb.append("perm=");
-      sb.append(stat.getOwner()).append(":");
-      sb.append(stat.getGroup()).append(":");
-      sb.append(stat.getPermission());
+    FileStatus status = null;
+    if (stat != null) {
+      Path symlink = stat.isSymlink() ? new Path(stat.getSymlink()) : null;
+      Path path = dst != null ? new Path(dst) : new Path(src);
+      status = new FileStatus(stat.getLen(), stat.isDir(),
+          stat.getReplication(), stat.getBlockSize(), stat.getModificationTime(),
+          stat.getAccessTime(), stat.getPermission(), stat.getOwner(),
+          stat.getGroup(), symlink, path);
+    }
+    for (AuditLogger logger : auditLoggers) {
+      logger.logAuditEvent(succeeded, ugi.toString(), addr,
+          cmd, src, dst, status);
     }
-    auditLog.info(sb);
   }
 
   /**
@@ -304,6 +306,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   final DelegationTokenSecretManager dtSecretManager;
   private final boolean alwaysUseDelegationTokensForTests;
   
+  // Tracks whether the default audit logger is the only configured audit
+  // logger; this allows isAuditEnabled() to return false in case the
+  // underlying logger is disabled, and avoid some unnecessary work.
+  private final boolean isDefaultAuditLogger;
+  private final List<AuditLogger> auditLoggers;
 
   /** The namespace tree. */
   FSDirectory dir;
@@ -536,14 +543,50 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       this.dtSecretManager = createDelegationTokenSecretManager(conf);
       this.dir = new FSDirectory(fsImage, this, conf);
       this.safeMode = new SafeModeInfo(conf);
-
+      this.auditLoggers = initAuditLoggers(conf);
+      this.isDefaultAuditLogger = auditLoggers.size() == 1 &&
+        auditLoggers.get(0) instanceof DefaultAuditLogger;
     } catch(IOException e) {
       LOG.error(getClass().getSimpleName() + " initialization failed.", e);
       close();
       throw e;
+    } catch (RuntimeException re) {
+      LOG.error(getClass().getSimpleName() + " initialization failed.", re);
+      close();
+      throw re;
     }
   }
 
+  private List<AuditLogger> initAuditLoggers(Configuration conf) {
+    // Initialize the custom access loggers if configured.
+    Collection<String> alClasses = conf.getStringCollection(DFS_NAMENODE_AUDIT_LOGGERS_KEY);
+    List<AuditLogger> auditLoggers = Lists.newArrayList();
+    if (alClasses != null && !alClasses.isEmpty()) {
+      for (String className : alClasses) {
+        try {
+          AuditLogger logger;
+          if (DFS_NAMENODE_DEFAULT_AUDIT_LOGGER_NAME.equals(className)) {
+            logger = new DefaultAuditLogger();
+          } else {
+            logger = (AuditLogger) Class.forName(className).newInstance();
+          }
+          logger.initialize(conf);
+          auditLoggers.add(logger);
+        } catch (RuntimeException re) {
+          throw re;
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+      }
+    }
+
+    // Make sure there is at least one logger installed.
+    if (auditLoggers.isEmpty()) {
+      auditLoggers.add(new DefaultAuditLogger());
+    }
+    return auditLoggers;
+  }
+
   void loadFSImage(StartupOption startOpt, FSImage fsImage, boolean haEnabled)
       throws IOException {
     // format before starting up if requested
@@ -1003,8 +1046,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       // start in active.
       return haEnabled;
     }
-  
-    return haContext.getState() instanceof StandbyState;
+
+    return HAServiceState.STANDBY == haContext.getState().getServiceState();
   }
 
   /**
@@ -1030,7 +1073,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     long totalInodes = this.dir.totalInodes();
     long totalBlocks = this.getBlocksTotal();
     out.println(totalInodes + " files and directories, " + totalBlocks
-        + " blocks = " + (totalInodes + totalBlocks) + " total");
+        + " blocks = " + (totalInodes + totalBlocks)
+        + " total filesystem objects");
 
     blockManager.metaSave(out);
   }
@@ -1076,7 +1120,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
       setPermissionInt(src, permission);
     } catch (AccessControlException e) {
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
                       getRemoteIp(),
                       "setPermission", src, null, null);
@@ -1098,14 +1142,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       }
       checkOwner(src);
       dir.setPermission(src, permission);
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         resultingStat = dir.getFileInfo(src, false);
       }
     } finally {
       writeUnlock();
     }
     getEditLog().logSync();
-    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+    if (isAuditEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
                     getRemoteIp(),
                     "setPermission", src, null, resultingStat);
@@ -1122,7 +1166,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
       setOwnerInt(src, username, group);
     } catch (AccessControlException e) {
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
                       getRemoteIp(),
                       "setOwner", src, null, null);
@@ -1153,14 +1197,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         }
       }
       dir.setOwner(src, username, group);
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         resultingStat = dir.getFileInfo(src, false);
       }
     } finally {
       writeUnlock();
     }
     getEditLog().logSync();
-    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+    if (isAuditEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
                     getRemoteIp(),
                     "setOwner", src, null, resultingStat);
@@ -1203,7 +1247,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       return getBlockLocationsInt(src, offset, length, doAccessTime,
                                   needBlockToken, checkSafeMode);
     } catch (AccessControlException e) {
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
                       getRemoteIp(),
                       "open", src, null, null);
@@ -1229,7 +1273,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
     final LocatedBlocks ret = getBlockLocationsUpdateTimes(src,
         offset, length, doAccessTime, needBlockToken);  
-    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+    if (isAuditEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
                     getRemoteIp(),
                     "open", src, null, null);
@@ -1310,7 +1354,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
       concatInt(target, srcs);
     } catch (AccessControlException e) {
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getLoginUser(),
                       getRemoteIp(),
                       "concat", Arrays.toString(srcs), target, null);
@@ -1353,14 +1397,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         throw new SafeModeException("Cannot concat " + target, safeMode);
       }
       concatInternal(target, srcs);
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         resultingStat = dir.getFileInfo(target, false);
       }
     } finally {
       writeUnlock();
     }
     getEditLog().logSync();
-    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+    if (isAuditEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getLoginUser(),
                     getRemoteIp(),
                     "concat", Arrays.toString(srcs), target, resultingStat);
@@ -1481,7 +1525,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
       setTimesInt(src, mtime, atime);
     } catch (AccessControlException e) {
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
                       getRemoteIp(),
                       "setTimes", src, null, null);
@@ -1507,7 +1551,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       INode inode = dir.getINode(src);
       if (inode != null) {
         dir.setTimes(src, inode, mtime, atime, true);
-        if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+        if (isAuditEnabled() && isExternalInvocation()) {
           final HdfsFileStatus stat = dir.getFileInfo(src, false);
           logAuditEvent(UserGroupInformation.getCurrentUser(),
                         getRemoteIp(),
@@ -1530,7 +1574,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
       createSymlinkInt(target, link, dirPerms, createParent);
     } catch (AccessControlException e) {
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
                       getRemoteIp(),
                       "createSymlink", link, target, null);
@@ -1551,14 +1595,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         verifyParentDir(link);
       }
       createSymlinkInternal(target, link, dirPerms, createParent);
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         resultingStat = dir.getFileInfo(link, false);
       }
     } finally {
       writeUnlock();
     }
     getEditLog().logSync();
-    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+    if (isAuditEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
                     getRemoteIp(),
                     "createSymlink", link, target, resultingStat);
@@ -1614,7 +1658,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
       return setReplicationInt(src, replication);
     } catch (AccessControlException e) {
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
                       getRemoteIp(),
                       "setReplication", src, null, null);
@@ -1650,7 +1694,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
 
     getEditLog().logSync();
-    if (isFile && auditLog.isInfoEnabled() && isExternalInvocation()) {
+    if (isFile && isAuditEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
                     getRemoteIp(),
                     "setReplication", src, null, null);
@@ -1706,7 +1750,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       startFileInt(src, permissions, holder, clientMachine, flag, createParent,
                    replication, blockSize);
     } catch (AccessControlException e) {
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
                       getRemoteIp(),
                       "create", src, null, null);
@@ -1739,7 +1783,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       }
     } 
 
-    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+    if (isAuditEnabled() && isExternalInvocation()) {
       final HdfsFileStatus stat = dir.getFileInfo(src, false);
       logAuditEvent(UserGroupInformation.getCurrentUser(),
                     getRemoteIp(),
@@ -2040,7 +2084,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
       return appendFileInt(src, holder, clientMachine);
     } catch (AccessControlException e) {
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
                       getRemoteIp(),
                       "append", src, null, null);
@@ -2086,7 +2130,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
             +" block size " + lb.getBlock().getNumBytes());
       }
     }
-    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+    if (isAuditEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
                     getRemoteIp(),
                     "append", src, null, null);
@@ -2532,7 +2576,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
       return renameToInt(src, dst);
     } catch (AccessControlException e) {
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
                       getRemoteIp(),
                       "rename", src, dst, null);
@@ -2554,14 +2598,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       checkOperation(OperationCategory.WRITE);
 
       status = renameToInternal(src, dst);
-      if (status && auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (status && isAuditEnabled() && isExternalInvocation()) {
         resultingStat = dir.getFileInfo(dst, false);
       }
     } finally {
       writeUnlock();
     }
     getEditLog().logSync();
-    if (status && auditLog.isInfoEnabled() && isExternalInvocation()) {
+    if (status && isAuditEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
                     getRemoteIp(),
                     "rename", src, dst, resultingStat);
@@ -2583,15 +2627,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     if (isPermissionEnabled) {
       //We should not be doing this.  This is move() not renameTo().
       //but for now,
+      //NOTE: yes, this is bad!  it's assuming much lower level behavior
+      //      of rewriting the dst
       String actualdst = dir.isDir(dst)?
           dst + Path.SEPARATOR + new Path(src).getName(): dst;
       checkParentAccess(src, FsAction.WRITE);
       checkAncestorAccess(actualdst, FsAction.WRITE);
     }
 
-    HdfsFileStatus dinfo = dir.getFileInfo(dst, false);
     if (dir.renameTo(src, dst)) {
-      unprotectedChangeLease(src, dst, dinfo);     // update lease with new filename
       return true;
     }
     return false;
@@ -2611,14 +2655,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       checkOperation(OperationCategory.WRITE);
 
       renameToInternal(src, dst, options);
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         resultingStat = dir.getFileInfo(dst, false); 
       }
     } finally {
       writeUnlock();
     }
     getEditLog().logSync();
-    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+    if (isAuditEnabled() && isExternalInvocation()) {
       StringBuilder cmd = new StringBuilder("rename options=");
       for (Rename option : options) {
         cmd.append(option.value()).append(" ");
@@ -2642,9 +2686,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       checkAncestorAccess(dst, FsAction.WRITE);
     }
 
-    HdfsFileStatus dinfo = dir.getFileInfo(dst, false);
     dir.renameTo(src, dst, options);
-    unprotectedChangeLease(src, dst, dinfo); // update lease with new filename
   }
   
   /**
@@ -2659,7 +2701,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
       return deleteInt(src, recursive);
     } catch (AccessControlException e) {
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
                       getRemoteIp(),
                       "delete", src, null, null);
@@ -2675,7 +2717,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       NameNode.stateChangeLog.debug("DIR* NameSystem.delete: " + src);
     }
     boolean status = deleteInternal(src, recursive, true);
-    if (status && auditLog.isInfoEnabled() && isExternalInvocation()) {
+    if (status && isAuditEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
                     getRemoteIp(),
                     "delete", src, null, null);
@@ -2841,7 +2883,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       }
       stat = dir.getFileInfo(src, resolveLink);
     } catch (AccessControlException e) {
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
                       getRemoteIp(),
                       "getfileinfo", src, null, null);
@@ -2850,7 +2892,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } finally {
       readUnlock();
     }
-    if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+    if (isAuditEnabled() && isExternalInvocation()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
                     getRemoteIp(),
                     "getfileinfo", src, null, null);
@@ -2866,7 +2908,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
       return mkdirsInt(src, permissions, createParent);
     } catch (AccessControlException e) {
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
                       getRemoteIp(),
                       "mkdirs", src, null, null);
@@ -2890,7 +2932,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       writeUnlock();
     }
     getEditLog().logSync();
-    if (status && auditLog.isInfoEnabled() && isExternalInvocation()) {
+    if (status && isAuditEnabled() && isExternalInvocation()) {
       final HdfsFileStatus stat = dir.getFileInfo(src, false);
       logAuditEvent(UserGroupInformation.getCurrentUser(),
                     getRemoteIp(),
@@ -2979,9 +3021,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /** Persist all metadata about this file.
    * @param src The string representation of the path
    * @param clientName The string representation of the client
+   * @param lastBlockLength The length of the last block 
+   *                        under construction reported from client.
    * @throws IOException if path does not exist
    */
-  void fsync(String src, String clientName) 
+  void fsync(String src, String clientName, long lastBlockLength) 
       throws IOException, UnresolvedLinkException {
     NameNode.stateChangeLog.info("BLOCK* fsync: " + src + " for " + clientName);
     writeLock();
@@ -2991,6 +3035,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         throw new SafeModeException("Cannot fsync file " + src, safeMode);
       }
       INodeFileUnderConstruction pendingFile  = checkLease(src, clientName);
+      if (lastBlockLength > 0) {
+        pendingFile.updateLengthOfLastBlock(lastBlockLength);
+      }
       dir.persistBlocks(src, pendingFile);
     } finally {
       writeUnlock();
@@ -3319,7 +3366,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
       return getListingInt(src, startAfter, needLocation);
     } catch (AccessControlException e) {
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         logAuditEvent(false, UserGroupInformation.getCurrentUser(),
                       getRemoteIp(),
                       "listStatus", src, null, null);
@@ -3343,7 +3390,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           checkTraverse(src);
         }
       }
-      if (auditLog.isInfoEnabled() && isExternalInvocation()) {
+      if (isAuditEnabled() && isExternalInvocation()) {
         logAuditEvent(UserGroupInformation.getCurrentUser(),
                       getRemoteIp(),
                       "listStatus", src, null, null);
@@ -3433,15 +3480,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   private NNHAStatusHeartbeat createHaStatusHeartbeat() {
     HAState state = haContext.getState();
-    NNHAStatusHeartbeat.State hbState;
-    if (state instanceof ActiveState) {
-      hbState = NNHAStatusHeartbeat.State.ACTIVE;
-    } else if (state instanceof StandbyState) {
-      hbState = NNHAStatusHeartbeat.State.STANDBY;      
-    } else {
-      throw new AssertionError("Invalid state: " + state.getClass());
-    }
-    return new NNHAStatusHeartbeat(hbState,
+    return new NNHAStatusHeartbeat(state.getServiceState(),
         getFSImage().getLastAppliedOrWrittenTxId());
   }
 
@@ -3870,7 +3909,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     private synchronized void leave() {
       // if not done yet, initialize replication queues.
       // In the standby, do not populate repl queues
-      if (!isPopulatingReplQueues() && !isInStandbyState()) {
+      if (!isPopulatingReplQueues() && shouldPopulateReplQueues()) {
         initializeReplQueues();
       }
       long timeInSafemode = now() - startTime;
@@ -3913,7 +3952,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
      * initializing replication queues.
      */
     private synchronized boolean canInitializeReplQueues() {
-      return !isInStandbyState() && blockSafe >= blockReplQueueThreshold;
+      return shouldPopulateReplQueues()
+          && blockSafe >= blockReplQueueThreshold;
     }
       
     /** 
@@ -4253,7 +4293,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   @Override
   public boolean isPopulatingReplQueues() {
-    if (isInStandbyState()) {
+    if (!shouldPopulateReplQueues()) {
       return false;
     }
     // safeMode is volatile, and may be set to null at any time
@@ -4262,7 +4302,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       return true;
     return safeMode.isPopulatingReplQueues();
   }
-    
+
+  private boolean shouldPopulateReplQueues() {
+    if(haContext == null || haContext.getState() == null)
+      return false;
+    return haContext.getState().shouldPopulateReplQueues();
+  }
+
   @Override
   public void incrementSafeBlockCount(int replication) {
     // safeMode is volatile, and may be set to null at any time
@@ -4880,31 +4926,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   // rename was successful. If any part of the renamed subtree had
   // files that were being written to, update with new filename.
-  void unprotectedChangeLease(String src, String dst, HdfsFileStatus dinfo) {
-    String overwrite;
-    String replaceBy;
+  void unprotectedChangeLease(String src, String dst) {
     assert hasWriteLock();
-
-    boolean destinationExisted = true;
-    if (dinfo == null) {
-      destinationExisted = false;
-    }
-
-    if (destinationExisted && dinfo.isDir()) {
-      Path spath = new Path(src);
-      Path parent = spath.getParent();
-      if (parent.isRoot()) {
-        overwrite = parent.toString();
-      } else {
-        overwrite = parent.toString() + Path.SEPARATOR;
-      }
-      replaceBy = dst + Path.SEPARATOR;
-    } else {
-      overwrite = src;
-      replaceBy = dst;
-    }
-
-    leaseManager.changeLease(src, dst, overwrite, replaceBy);
+    leaseManager.changeLease(src, dst);
   }
 
   /**
@@ -4915,19 +4939,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     // lock on our behalf. If we took the read lock here, we could block
     // for fairness if a writer is waiting on the lock.
     synchronized (leaseManager) {
-      out.writeInt(leaseManager.countPath()); // write the size
-
-      for (Lease lease : leaseManager.getSortedLeases()) {
-        for(String path : lease.getPaths()) {
-          // verify that path exists in namespace
-          final INodeFileUnderConstruction cons;
-          try {
-            cons = INodeFileUnderConstruction.valueOf(dir.getINode(path), path);
-          } catch (UnresolvedLinkException e) {
-            throw new AssertionError("Lease files should reside on this FS");
-          }
-          FSImageSerialization.writeINodeUnderConstruction(out, cons, path);
-        }
+      Map<String, INodeFileUnderConstruction> nodes =
+          leaseManager.getINodesUnderConstruction();
+      out.writeInt(nodes.size()); // write the size    
+      for (Map.Entry<String, INodeFileUnderConstruction> entry
+           : nodes.entrySet()) {
+        FSImageSerialization.writeINodeUnderConstruction(
+            out, entry.getValue(), entry.getKey());
       }
     }
   }
@@ -5286,7 +5304,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * Log fsck event in the audit log 
    */
   void logFsckEvent(String src, InetAddress remoteAddress) throws IOException {
-    if (auditLog.isInfoEnabled()) {
+    if (isAuditEnabled()) {
       logAuditEvent(UserGroupInformation.getCurrentUser(),
                     remoteAddress,
                     "fsck", src, null, null);
@@ -5541,4 +5559,44 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return this.blockManager.getDatanodeManager()
         .isAvoidingStaleDataNodesForWrite();
   }
+
+  /**
+   * Default AuditLogger implementation; used when no access logger is
+   * defined in the config file. It can also be explicitly listed in the
+   * config file.
+   */
+  private static class DefaultAuditLogger implements AuditLogger {
+
+    @Override
+    public void initialize(Configuration conf) {
+      // Nothing to do.
+    }
+
+    @Override
+    public void logAuditEvent(boolean succeeded, String userName,
+        InetAddress addr, String cmd, String src, String dst,
+        FileStatus status) {
+      if (auditLog.isInfoEnabled()) {
+        final StringBuilder sb = auditBuffer.get();
+        sb.setLength(0);
+        sb.append("allowed=").append(succeeded).append("\t");
+        sb.append("ugi=").append(userName).append("\t");
+        sb.append("ip=").append(addr).append("\t");
+        sb.append("cmd=").append(cmd).append("\t");
+        sb.append("src=").append(src).append("\t");
+        sb.append("dst=").append(dst).append("\t");
+        if (null == status) {
+          sb.append("perm=null");
+        } else {
+          sb.append("perm=");
+          sb.append(status.getOwner()).append(":");
+          sb.append(status.getGroup()).append(":");
+          sb.append(status.getPermission());
+        }
+        auditLog.info(sb);
+      }
+    }
+
+  }
+
 }

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

@@ -73,6 +73,11 @@ class INodeDirectory extends INode {
   INodeDirectory(INodeDirectory other) {
     super(other);
     this.children = other.children;
+    if (this.children != null) {
+      for (INode child : children) {
+        child.parent = this;
+      }
+    }
   }
   
   /** @return true unconditionally. */
@@ -106,6 +111,7 @@ class INodeDirectory extends INode {
 
     final int low = searchChildren(newChild);
     if (low>=0) { // an old child exists so replace by the newChild
+      children.get(low).parent = null;
       children.set(low, newChild);
     } else {
       throw new IllegalArgumentException("No child exists to be replaced");

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

@@ -171,4 +171,22 @@ class INodeFileUnderConstruction extends INodeFile implements MutableBlockCollec
     setBlock(numBlocks()-1, ucBlock);
     return ucBlock;
   }
+
+  /**
+   * Update the length for the last block
+   * 
+   * @param lastBlockLength
+   *          The length of the last block reported from client
+   * @throws IOException
+   */
+  void updateLengthOfLastBlock(long lastBlockLength) throws IOException {
+    BlockInfo lastBlock = this.getLastBlock();
+    assert (lastBlock != null) : "The last block for path "
+        + this.getFullPathName() + " is null when updating its length";
+    assert (lastBlock instanceof BlockInfoUnderConstruction) : "The last block for path "
+        + this.getFullPathName()
+        + " is not a BlockInfoUnderConstruction when updating its length";
+    lastBlock.setNumBytes(lastBlockLength);
+  }
+  
 }

+ 35 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java

@@ -17,9 +17,12 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.util.Time.now;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.SortedMap;
@@ -39,8 +42,6 @@ import org.apache.hadoop.util.Daemon;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
-import static org.apache.hadoop.util.Time.now;
-
 /**
  * LeaseManager does the lease housekeeping for writing on files.   
  * This class also provides useful static methods for lease recovery.
@@ -330,21 +331,19 @@ public class LeaseManager {
     }
   }
 
-  synchronized void changeLease(String src, String dst,
-      String overwrite, String replaceBy) {
+  synchronized void changeLease(String src, String dst) {
     if (LOG.isDebugEnabled()) {
       LOG.debug(getClass().getSimpleName() + ".changelease: " +
-               " src=" + src + ", dest=" + dst + 
-               ", overwrite=" + overwrite +
-               ", replaceBy=" + replaceBy);
+               " src=" + src + ", dest=" + dst);
     }
 
-    final int len = overwrite.length();
-    for(Map.Entry<String, Lease> entry : findLeaseWithPrefixPath(src, sortedLeasesByPath)) {
+    final int len = src.length();
+    for(Map.Entry<String, Lease> entry
+        : findLeaseWithPrefixPath(src, sortedLeasesByPath).entrySet()) {
       final String oldpath = entry.getKey();
       final Lease lease = entry.getValue();
-      //overwrite must be a prefix of oldpath
-      final String newpath = replaceBy + oldpath.substring(len);
+      // replace stem of src with new destination
+      final String newpath = dst + oldpath.substring(len);
       if (LOG.isDebugEnabled()) {
         LOG.debug("changeLease: replacing " + oldpath + " with " + newpath);
       }
@@ -355,7 +354,8 @@ public class LeaseManager {
   }
 
   synchronized void removeLeaseWithPrefixPath(String prefix) {
-    for(Map.Entry<String, Lease> entry : findLeaseWithPrefixPath(prefix, sortedLeasesByPath)) {
+    for(Map.Entry<String, Lease> entry
+        : findLeaseWithPrefixPath(prefix, sortedLeasesByPath).entrySet()) {
       if (LOG.isDebugEnabled()) {
         LOG.debug(LeaseManager.class.getSimpleName()
             + ".removeLeaseWithPrefixPath: entry=" + entry);
@@ -364,13 +364,13 @@ public class LeaseManager {
     }
   }
 
-  static private List<Map.Entry<String, Lease>> findLeaseWithPrefixPath(
+  static private Map<String, Lease> findLeaseWithPrefixPath(
       String prefix, SortedMap<String, Lease> path2lease) {
     if (LOG.isDebugEnabled()) {
       LOG.debug(LeaseManager.class.getSimpleName() + ".findLease: prefix=" + prefix);
     }
 
-    List<Map.Entry<String, Lease>> entries = new ArrayList<Map.Entry<String, Lease>>();
+    final Map<String, Lease> entries = new HashMap<String, Lease>();
     final int srclen = prefix.length();
 
     for(Map.Entry<String, Lease> entry : path2lease.tailMap(prefix).entrySet()) {
@@ -379,7 +379,7 @@ public class LeaseManager {
         return entries;
       }
       if (p.length() == srclen || p.charAt(srclen) == Path.SEPARATOR_CHAR) {
-        entries.add(entry);
+        entries.put(entry.getKey(), entry.getValue());
       }
     }
     return entries;
@@ -426,6 +426,26 @@ public class LeaseManager {
     }
   }
 
+  /**
+   * Get the list of inodes corresponding to valid leases.
+   * @return list of inodes
+   * @throws UnresolvedLinkException
+   */
+  Map<String, INodeFileUnderConstruction> getINodesUnderConstruction() {
+    Map<String, INodeFileUnderConstruction> inodes =
+        new TreeMap<String, INodeFileUnderConstruction>();
+    for (String p : sortedLeasesByPath.keySet()) {
+      // verify that path exists in namespace
+      try {
+        INode node = fsnamesystem.dir.getINode(p);
+        inodes.put(p, INodeFileUnderConstruction.valueOf(node, p));
+      } catch (IOException ioe) {
+        LOG.error(ioe);
+      }
+    }
+    return inodes;
+  }
+  
   /** Check the leases beginning from the oldest.
    *  @return true is sync is needed.
    */

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

@@ -598,11 +598,7 @@ public class NameNode {
     String nsId = getNameServiceId(conf);
     String namenodeId = HAUtil.getNameNodeId(conf, nsId);
     this.haEnabled = HAUtil.isHAEnabled(conf, nsId);
-    if (!haEnabled) {
-      state = ACTIVE_STATE;
-    } else {
-      state = STANDBY_STATE;
-    }
+    state = createHAState();
     this.allowStaleStandbyReads = HAUtil.shouldAllowStandbyReads(conf);
     this.haContext = createHAContext();
     try {
@@ -619,6 +615,10 @@ public class NameNode {
     }
   }
 
+  protected HAState createHAState() {
+    return !haEnabled ? ACTIVE_STATE : STANDBY_STATE;
+  }
+
   protected HAContext createHAContext() {
     return new NameNodeHAContext();
   }
@@ -1298,7 +1298,7 @@ public class NameNode {
    *          before exit.
    * @throws ExitException thrown only for testing.
    */
-  private synchronized void doImmediateShutdown(Throwable t)
+  protected synchronized void doImmediateShutdown(Throwable t)
       throws ExitException {
     String message = "Error encountered requiring NN shutdown. " +
         "Shutting down immediately.";

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -825,8 +825,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
   
   @Override // ClientProtocol
-  public void fsync(String src, String clientName) throws IOException {
-    namesystem.fsync(src, clientName);
+  public void fsync(String src, String clientName, long lastBlockLength)
+      throws IOException {
+    namesystem.fsync(src, clientName, lastBlockLength);
   }
 
   @Override // ClientProtocol

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

@@ -102,7 +102,7 @@ class NamenodeJspHelper {
     long usedNonHeap = (totalNonHeap * 100) / commitedNonHeap;
 
     String str = "<div>" + inodes + " files and directories, " + blocks + " blocks = "
-        + (inodes + blocks) + " total";
+        + (inodes + blocks) + " total filesystem objects";
     if (maxobjects != 0) {
       long pct = ((inodes + blocks) * 100) / maxobjects;
       str += " / " + maxobjects + " (" + pct + "%)";

+ 4 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NNHAStatusHeartbeat.java

@@ -19,31 +19,26 @@ package org.apache.hadoop.hdfs.server.protocol;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class NNHAStatusHeartbeat {
 
-  private State state;
+  private HAServiceState state;
   private long txid = HdfsConstants.INVALID_TXID;
   
-  public NNHAStatusHeartbeat(State state, long txid) {
+  public NNHAStatusHeartbeat(HAServiceState state, long txid) {
     this.state = state;
     this.txid = txid;
   }
 
-  public State getState() {
+  public HAServiceState getState() {
     return state;
   }
   
   public long getTxId() {
     return txid;
   }
-  
-  @InterfaceAudience.Private
-  public enum State {
-    ACTIVE,
-    STANDBY;
-  }
 }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -357,6 +357,7 @@ message SetQuotaResponseProto { // void response
 message FsyncRequestProto {
   required string src = 1;
   required string client = 2;
+  optional sint64 lastBlockLength = 3 [default = -1];
 }
 
 message FsyncResponseProto { // void response

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

@@ -1184,4 +1184,17 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.namenode.audit.loggers</name>
+  <value>default</value>
+  <description>
+    List of classes implementing audit loggers that will receive audit events.
+    These should be implementations of org.apache.hadoop.hdfs.server.namenode.AuditLogger.
+    The special value "default" can be used to reference the default audit
+    logger, which uses the configured log system. Installing custom audit loggers
+    may affect the performance and stability of the NameNode. Refer to the custom
+    logger's documentation for more details.
+  </description>
+</property>
+
 </configuration>

+ 183 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestVolumeId.java

@@ -0,0 +1,183 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class TestVolumeId {
+
+  @Test
+  public void testEquality() {
+    final VolumeId id1 = new HdfsVolumeId(new byte[] { (byte)0, (byte)0 });
+    testEq(true, id1, id1);
+
+    final VolumeId id2 = new HdfsVolumeId(new byte[] { (byte)0, (byte)1 });
+    testEq(true, id2, id2);
+    testEq(false, id1, id2);
+
+    final VolumeId id3 = new HdfsVolumeId(new byte[] { (byte)1, (byte)0 });
+    testEq(true, id3, id3);
+    testEq(false, id1, id3);
+    
+    // same as 2, but "invalid":
+    final VolumeId id2copy1 = new HdfsVolumeId(new byte[] { (byte)0, (byte)1 });
+    
+    testEq(true, id2, id2copy1);
+
+    // same as 2copy1: 
+    final VolumeId id2copy2 = new HdfsVolumeId(new byte[] { (byte)0, (byte)1 });
+    
+    testEq(true, id2, id2copy2);
+    
+    testEqMany(true, new VolumeId[] { id2, id2copy1, id2copy2 });
+    
+    testEqMany(false, new VolumeId[] { id1, id2, id3 });
+  }
+  
+  @SuppressWarnings("unchecked")
+  private <T> void testEq(final boolean eq, Comparable<? super T> id1, Comparable<? super T> id2) {
+    final int h1 = id1.hashCode();
+    final int h2 = id2.hashCode();
+    
+    // eq reflectivity:
+    assertTrue(id1.equals(id1));
+    assertTrue(id2.equals(id2));
+    assertEquals(0, id1.compareTo((T)id1));
+    assertEquals(0, id2.compareTo((T)id2));
+
+    // eq symmetry:
+    assertEquals(eq, id1.equals(id2));
+    assertEquals(eq, id2.equals(id1));
+    
+    // null comparison:
+    assertFalse(id1.equals(null));
+    assertFalse(id2.equals(null));
+    
+    // compareTo:
+    assertEquals(eq, 0 == id1.compareTo((T)id2));
+    assertEquals(eq, 0 == id2.compareTo((T)id1));
+    // compareTo must be antisymmetric:
+    assertEquals(sign(id1.compareTo((T)id2)), -sign(id2.compareTo((T)id1)));
+    
+    // compare with null should never return 0 to be consistent with #equals(): 
+    assertTrue(id1.compareTo(null) != 0);
+    assertTrue(id2.compareTo(null) != 0);
+    
+    // check that hash codes did not change:
+    assertEquals(h1, id1.hashCode());
+    assertEquals(h2, id2.hashCode());
+    if (eq) {
+      // in this case the hash codes must be the same:
+      assertEquals(h1, h2);
+    }
+  }
+  
+  private static int sign(int x) {
+    if (x == 0) {
+      return 0;
+    } else if (x > 0) {
+      return 1;
+    } else {
+      return -1;
+    }
+  }
+  
+  @SuppressWarnings("unchecked")
+  private <T> void testEqMany(final boolean eq, Comparable<? super T>... volumeIds) {
+    Comparable<? super T> vidNext;
+    int sum = 0;
+    for (int i=0; i<volumeIds.length; i++) {
+      if (i == volumeIds.length - 1) {
+        vidNext = volumeIds[0];
+      } else {
+        vidNext = volumeIds[i + 1];
+      }
+      testEq(eq, volumeIds[i], vidNext);
+      sum += sign(volumeIds[i].compareTo((T)vidNext));
+    }
+    // the comparison relationship must always be acyclic:
+    assertTrue(sum < volumeIds.length);
+  }
+
+  /*
+   * Test HdfsVolumeId(new byte[0]) instances: show that we permit such
+   * objects, they are still valid, and obey the same equality
+   * rules other objects do. 
+   */
+  @Test
+  public void testIdEmptyBytes() {
+    final VolumeId idEmpty1   = new HdfsVolumeId(new byte[0]);
+    assertTrue(idEmpty1.isValid());
+    final VolumeId idEmpty2   = new HdfsVolumeId(new byte[0]);
+    assertTrue(idEmpty2.isValid());
+    final VolumeId idNotEmpty = new HdfsVolumeId(new byte[] { (byte)1 });
+    assertTrue(idNotEmpty.isValid());
+    
+    testEq(true, idEmpty1, idEmpty2);
+    testEq(false, idEmpty1, idNotEmpty);
+    testEq(false, idEmpty2, idNotEmpty);
+  }
+  
+  /*
+   * Test the VolumeId.INVALID_VOLUME_ID singleton.
+   */
+  @Test
+  public void testInvalidId() {
+    try {
+      new HdfsVolumeId(null);
+      assertTrue("NPE expected.", false);
+    } catch (NullPointerException npe) {
+      // okay
+    }
+    final VolumeId idEmpty   = new HdfsVolumeId(new byte[] {});
+    final VolumeId idNotEmpty = new HdfsVolumeId(new byte[] { (byte)1 });
+    
+    testEq(false, VolumeId.INVALID_VOLUME_ID, idNotEmpty);
+    testEq(false, VolumeId.INVALID_VOLUME_ID, idEmpty);
+    
+    testEqMany(true, 
+        new VolumeId[] { 
+          VolumeId.INVALID_VOLUME_ID, 
+          VolumeId.INVALID_VOLUME_ID, 
+          VolumeId.INVALID_VOLUME_ID } );
+    testEqMany(false, 
+        new VolumeId[] {
+          VolumeId.INVALID_VOLUME_ID, 
+          idEmpty, 
+          idNotEmpty });
+  }
+  
+  /*
+   * test #toString() for typical VolumeId equality classes
+   */
+  @Test
+  public void testToString() {
+    // The #toString() return value is only checked for != null.
+    // We cannot assert more.
+    String strInvalid = VolumeId.INVALID_VOLUME_ID.toString();
+    assertNotNull(strInvalid);
+    
+    String strEmpty = new HdfsVolumeId(new byte[] {}).toString();
+    assertNotNull(strEmpty);
+    
+    String strNotEmpty = new HdfsVolumeId(new byte[] { (byte)1 }).toString();
+    assertNotNull(strNotEmpty);
+  } 
+  
+}

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

@@ -321,7 +321,7 @@ public class MiniDFSCluster {
   /**
    * Used by builder to create and return an instance of MiniDFSCluster
    */
-  private MiniDFSCluster(Builder builder) throws IOException {
+  protected MiniDFSCluster(Builder builder) throws IOException {
     if (builder.nnTopology == null) {
       // If no topology is specified, build a single NN. 
       builder.nnTopology = MiniDFSNNTopology.simpleSingleNN(
@@ -369,8 +369,8 @@ public class MiniDFSCluster {
 
   private Configuration conf;
   private NameNodeInfo[] nameNodes;
-  private int numDataNodes;
-  private ArrayList<DataNodeProperties> dataNodes = 
+  protected int numDataNodes;
+  protected List<DataNodeProperties> dataNodes = 
                          new ArrayList<DataNodeProperties>();
   private File base_dir;
   private File data_dir;
@@ -2303,7 +2303,7 @@ public class MiniDFSCluster {
     return port;
   }
   
-  private void setupDatanodeAddress(Configuration conf, boolean setupHostsFile,
+  protected void setupDatanodeAddress(Configuration conf, boolean setupHostsFile,
                            boolean checkDataNodeAddrConfig) throws IOException {
     if (setupHostsFile) {
       String hostsFile = conf.get(DFS_HOSTS, "").trim();

+ 228 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java

@@ -0,0 +1,228 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY;
+import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter;
+import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.net.StaticMapping;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.ssl.SSLFactory;
+
+public class MiniDFSClusterWithNodeGroup extends MiniDFSCluster {
+
+  private static String[] NODE_GROUPS = null;
+  private static final Log LOG = LogFactory.getLog(MiniDFSClusterWithNodeGroup.class);
+  
+  public MiniDFSClusterWithNodeGroup(Builder builder) throws IOException {
+    super(builder);
+  }
+
+  public static void setNodeGroups (String[] nodeGroups) {
+    NODE_GROUPS = nodeGroups;
+  }
+
+  public synchronized void startDataNodes(Configuration conf, int numDataNodes,
+      boolean manageDfsDirs, StartupOption operation, 
+      String[] racks, String[] nodeGroups, String[] hosts,
+      long[] simulatedCapacities,
+      boolean setupHostsFile,
+      boolean checkDataNodeAddrConfig,
+      boolean checkDataNodeHostConfig) throws IOException {
+    if (operation == StartupOption.RECOVER) {
+      return;
+    }
+    if (checkDataNodeHostConfig) {
+      conf.setIfUnset(DFS_DATANODE_HOST_NAME_KEY, "127.0.0.1");
+    } else {
+      conf.set(DFS_DATANODE_HOST_NAME_KEY, "127.0.0.1");
+    }
+    conf.set(DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY, "127.0.0.1");
+
+    int curDatanodesNum = dataNodes.size();
+    // for mincluster's the default initialDelay for BRs is 0
+    if (conf.get(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY) == null) {
+      conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY, 0);
+    }
+    // If minicluster's name node is null assume that the conf has been
+    // set with the right address:port of the name node.
+    //
+    if (racks != null && numDataNodes > racks.length ) {
+      throw new IllegalArgumentException( "The length of racks [" + racks.length
+          + "] is less than the number of datanodes [" + numDataNodes + "].");
+    }
+
+    if (nodeGroups != null && numDataNodes > nodeGroups.length ) {
+      throw new IllegalArgumentException( "The length of nodeGroups [" + nodeGroups.length
+          + "] is less than the number of datanodes [" + numDataNodes + "].");
+    }
+
+    if (hosts != null && numDataNodes > hosts.length ) {
+      throw new IllegalArgumentException( "The length of hosts [" + hosts.length
+          + "] is less than the number of datanodes [" + numDataNodes + "].");
+    }
+    //Generate some hostnames if required
+    if (racks != null && hosts == null) {
+      hosts = new String[numDataNodes];
+      for (int i = curDatanodesNum; i < curDatanodesNum + numDataNodes; i++) {
+        hosts[i - curDatanodesNum] = "host" + i + ".foo.com";
+      }
+    }
+
+    if (simulatedCapacities != null 
+        && numDataNodes > simulatedCapacities.length) {
+      throw new IllegalArgumentException( "The length of simulatedCapacities [" 
+          + simulatedCapacities.length
+          + "] is less than the number of datanodes [" + numDataNodes + "].");
+    }
+
+    String [] dnArgs = (operation == null ||
+    operation != StartupOption.ROLLBACK) ?
+        null : new String[] {operation.getName()};
+
+    for (int i = curDatanodesNum; i < curDatanodesNum+numDataNodes; i++) {
+      Configuration dnConf = new HdfsConfiguration(conf);
+      // Set up datanode address
+      setupDatanodeAddress(dnConf, setupHostsFile, checkDataNodeAddrConfig);
+      if (manageDfsDirs) {
+        File dir1 = getInstanceStorageDir(i, 0);
+        File dir2 = getInstanceStorageDir(i, 1);
+        dir1.mkdirs();
+        dir2.mkdirs();
+        if (!dir1.isDirectory() || !dir2.isDirectory()) { 
+          throw new IOException("Mkdirs failed to create directory for DataNode "
+              + i + ": " + dir1 + " or " + dir2);
+        }
+        String dirs = fileAsURI(dir1) + "," + fileAsURI(dir2);
+        dnConf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dirs);
+        conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dirs);
+      }
+      if (simulatedCapacities != null) {
+        SimulatedFSDataset.setFactory(dnConf);
+        dnConf.setLong(SimulatedFSDataset.CONFIG_PROPERTY_CAPACITY,
+        simulatedCapacities[i-curDatanodesNum]);
+      }
+      LOG.info("Starting DataNode " + i + " with "
+          + DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY + ": "
+          + dnConf.get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY));
+      if (hosts != null) {
+        dnConf.set(DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY, hosts[i - curDatanodesNum]);
+        LOG.info("Starting DataNode " + i + " with hostname set to: "
+            + dnConf.get(DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY));
+      }
+      if (racks != null) {
+        String name = hosts[i - curDatanodesNum];
+        if (nodeGroups == null) {
+          LOG.info("Adding node with hostname : " + name + " to rack " +
+             racks[i-curDatanodesNum]);
+          StaticMapping.addNodeToRack(name,racks[i-curDatanodesNum]);
+        } else {
+          LOG.info("Adding node with hostname : " + name + " to serverGroup " +
+              nodeGroups[i-curDatanodesNum] + " and rack " +
+              racks[i-curDatanodesNum]);
+          StaticMapping.addNodeToRack(name,racks[i-curDatanodesNum] + 
+              nodeGroups[i-curDatanodesNum]);
+        }
+      }
+      Configuration newconf = new HdfsConfiguration(dnConf); // save config
+      if (hosts != null) {
+        NetUtils.addStaticResolution(hosts[i - curDatanodesNum], "localhost");
+      }
+      
+      SecureResources secureResources = null;
+      if (UserGroupInformation.isSecurityEnabled()) {
+        SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, dnConf);
+        try {
+          secureResources = SecureDataNodeStarter.getSecureResources(sslFactory, dnConf);
+        } catch (Exception ex) {
+          ex.printStackTrace();
+        }
+      }
+      DataNode dn = DataNode.instantiateDataNode(dnArgs, dnConf, secureResources);
+      if(dn == null)
+        throw new IOException("Cannot start DataNode in "
+          + dnConf.get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY));
+      //since the HDFS does things based on IP:port, we need to add the mapping
+      //for IP:port to rackId
+      String ipAddr = dn.getXferAddress().getAddress().getHostAddress();
+      if (racks != null) {
+        int port = dn.getXferAddress().getPort();
+        if (nodeGroups == null) {
+          LOG.info("Adding node with IP:port : " + ipAddr + ":" + port +
+              " to rack " + racks[i-curDatanodesNum]);
+          StaticMapping.addNodeToRack(ipAddr + ":" + port,
+              racks[i-curDatanodesNum]);
+        } else {
+          LOG.info("Adding node with IP:port : " + ipAddr + ":" + port + " to nodeGroup " +
+          nodeGroups[i-curDatanodesNum] + " and rack " + racks[i-curDatanodesNum]);
+          StaticMapping.addNodeToRack(ipAddr + ":" + port, racks[i-curDatanodesNum] + 
+              nodeGroups[i-curDatanodesNum]);
+        }
+      }
+      dn.runDatanodeDaemon();
+      dataNodes.add(new DataNodeProperties(dn, newconf, dnArgs, secureResources));
+    }
+    curDatanodesNum += numDataNodes;
+    this.numDataNodes += numDataNodes;
+    waitActive();
+  }
+
+  public synchronized void startDataNodes(Configuration conf, int numDataNodes, 
+      boolean manageDfsDirs, StartupOption operation, 
+      String[] racks, String[] nodeGroups, String[] hosts,
+      long[] simulatedCapacities,
+      boolean setupHostsFile) throws IOException {
+    startDataNodes(conf, numDataNodes, manageDfsDirs, operation, racks, nodeGroups, 
+        hosts, simulatedCapacities, setupHostsFile, false, false);
+  }
+
+  public void startDataNodes(Configuration conf, int numDataNodes, 
+      boolean manageDfsDirs, StartupOption operation, 
+      String[] racks, long[] simulatedCapacities,
+      String[] nodeGroups) throws IOException {
+    startDataNodes(conf, numDataNodes, manageDfsDirs, operation, racks, nodeGroups,
+        null, simulatedCapacities, false);
+  }
+
+  // This is for initialize from parent class.
+  @Override
+  public synchronized void startDataNodes(Configuration conf, int numDataNodes, 
+      boolean manageDfsDirs, StartupOption operation, 
+      String[] racks, String[] hosts,
+      long[] simulatedCapacities,
+      boolean setupHostsFile,
+      boolean checkDataNodeAddrConfig,
+      boolean checkDataNodeHostConfig) throws IOException {
+    startDataNodes(conf, numDataNodes, manageDfsDirs, operation, racks, 
+        NODE_GROUPS, hosts, simulatedCapacities, setupHostsFile, 
+        checkDataNodeAddrConfig, checkDataNodeHostConfig);
+  }
+
+}

+ 128 - 23
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHFlush.java

@@ -23,12 +23,14 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;
+import java.util.EnumSet;
 
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.log4j.Level;
 import org.junit.Test;
@@ -43,16 +45,21 @@ public class TestHFlush {
   
   private final String fName = "hflushtest.dat";
   
-  /** The test uses {@link #doTheJob(Configuration, String, long, short)
+  /**
+   * The test uses
+   * {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)} 
    * to write a file with a standard block size
    */
   @Test
   public void hFlush_01() throws IOException {
-    doTheJob(new HdfsConfiguration(), fName, AppendTestUtil.BLOCK_SIZE, (short)2);
+    doTheJob(new HdfsConfiguration(), fName, AppendTestUtil.BLOCK_SIZE,
+        (short) 2, false, EnumSet.noneOf(SyncFlag.class));
   }
 
-  /** The test uses {@link #doTheJob(Configuration, String, long, short)
-   * to write a file with a custom block size so the writes will be
+  /**
+   * The test uses
+   * {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)} 
+   * to write a file with a custom block size so the writes will be 
    * happening across block' boundaries
    */
   @Test
@@ -64,14 +71,17 @@ public class TestHFlush {
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
 
-    doTheJob(conf, fName, customBlockSize, (short)2);
+    doTheJob(conf, fName, customBlockSize, (short) 2, false,
+        EnumSet.noneOf(SyncFlag.class));
   }
 
-  /** The test uses {@link #doTheJob(Configuration, String, long, short)
-   * to write a file with a custom block size so the writes will be
+  /**
+   * The test uses
+   * {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)} 
+   * to write a file with a custom block size so the writes will be 
    * happening across block's and checksum' boundaries
    */
- @Test
+  @Test
   public void hFlush_03() throws IOException {
     Configuration conf = new HdfsConfiguration();
     int customPerChecksumSize = 400;
@@ -80,22 +90,106 @@ public class TestHFlush {
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
 
-    doTheJob(conf, fName, customBlockSize, (short)2);
+    doTheJob(conf, fName, customBlockSize, (short) 2, false,
+        EnumSet.noneOf(SyncFlag.class));
+  }
+
+  /**
+   * Test hsync (with updating block length in NameNode) while no data is
+   * actually written yet
+   */
+  @Test
+  public void hSyncUpdateLength_00() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
+        2).build();
+    DistributedFileSystem fileSystem =
+        (DistributedFileSystem)cluster.getFileSystem();
+    
+    try {
+      Path path = new Path(fName);
+      FSDataOutputStream stm = fileSystem.create(path, true, 4096, (short) 2,
+          AppendTestUtil.BLOCK_SIZE);
+      System.out.println("Created file " + path.toString());
+      ((DFSOutputStream) stm.getWrappedStream()).hsync(EnumSet
+          .of(SyncFlag.UPDATE_LENGTH));
+      long currentFileLength = fileSystem.getFileStatus(path).getLen();
+      assertEquals(0L, currentFileLength);
+      stm.close();
+    } finally {
+      fileSystem.close();
+      cluster.shutdown();
+    }
+  }
+  
+  /**
+   * The test calls
+   * {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)}
+   * while requiring the semantic of {@link SyncFlag#UPDATE_LENGTH}.
+   */
+  @Test
+  public void hSyncUpdateLength_01() throws IOException {
+    doTheJob(new HdfsConfiguration(), fName, AppendTestUtil.BLOCK_SIZE,
+        (short) 2, true, EnumSet.of(SyncFlag.UPDATE_LENGTH));
   }
 
   /**
-    The method starts new cluster with defined Configuration;
-    creates a file with specified block_size and writes 10 equal sections in it;
-    it also calls hflush() after each write and throws an IOException in case of 
-    an error.
-    @param conf cluster configuration
-    @param fileName of the file to be created and processed as required
-    @param block_size value to be used for the file's creation
-    @param replicas is the number of replicas
-    @throws IOException in case of any errors 
+   * The test calls
+   * {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)}
+   * while requiring the semantic of {@link SyncFlag#UPDATE_LENGTH}.
+   * Similar with {@link #hFlush_02()} , it writes a file with a custom block
+   * size so the writes will be happening across block' boundaries
+   */
+  @Test
+  public void hSyncUpdateLength_02() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    int customPerChecksumSize = 512;
+    int customBlockSize = customPerChecksumSize * 3;
+    // Modify defaul filesystem settings
+    conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
+
+    doTheJob(conf, fName, customBlockSize, (short) 2, true,
+        EnumSet.of(SyncFlag.UPDATE_LENGTH));
+  }
+  
+  /**
+   * The test calls
+   * {@link #doTheJob(Configuration, String, long, short, boolean, EnumSet)}
+   * while requiring the semantic of {@link SyncFlag#UPDATE_LENGTH}.
+   * Similar with {@link #hFlush_03()} , it writes a file with a custom block
+   * size so the writes will be happening across block's and checksum'
+   * boundaries.
+   */
+  @Test
+  public void hSyncUpdateLength_03() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    int customPerChecksumSize = 400;
+    int customBlockSize = customPerChecksumSize * 3;
+    // Modify defaul filesystem settings
+    conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, customPerChecksumSize);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, customBlockSize);
+
+    doTheJob(conf, fName, customBlockSize, (short) 2, true,
+        EnumSet.of(SyncFlag.UPDATE_LENGTH));
+  }
+  
+  /**
+   * The method starts new cluster with defined Configuration; creates a file
+   * with specified block_size and writes 10 equal sections in it; it also calls
+   * hflush/hsync after each write and throws an IOException in case of an error.
+   * 
+   * @param conf cluster configuration
+   * @param fileName of the file to be created and processed as required
+   * @param block_size value to be used for the file's creation
+   * @param replicas is the number of replicas
+   * @param isSync hsync or hflush         
+   * @param syncFlags specify the semantic of the sync/flush
+   * @throws IOException in case of any errors
    */
   public static void doTheJob(Configuration conf, final String fileName,
-                              long block_size, short replicas) throws IOException {
+      long block_size, short replicas, boolean isSync,
+      EnumSet<SyncFlag> syncFlags) throws IOException {
     byte[] fileContent;
     final int SECTIONS = 10;
 
@@ -119,8 +213,21 @@ public class TestHFlush {
         System.out.println("Writing " + (tenth * i) + " to " + (tenth * (i+1)) + " section to file " + fileName);
         // write to the file
         stm.write(fileContent, tenth * i, tenth);
-        // Wait while hflush() pushes all packets through built pipeline
-        ((DFSOutputStream)stm.getWrappedStream()).hflush();
+        
+        // Wait while hflush/hsync pushes all packets through built pipeline
+        if (isSync) {
+          ((DFSOutputStream)stm.getWrappedStream()).hsync(syncFlags);
+        } else {
+          ((DFSOutputStream)stm.getWrappedStream()).hflush();
+        }
+        
+        // Check file length if updatelength is required
+        if (isSync && syncFlags.contains(SyncFlag.UPDATE_LENGTH)) {
+          long currentFileLength = fileSystem.getFileStatus(path).getLen();
+          assertEquals(
+            "File size doesn't match for hsync/hflush with updating the length",
+            tenth * (i + 1), currentFileLength);
+        }
         byte [] toRead = new byte[tenth];
         byte [] expected = new byte[tenth];
         System.arraycopy(fileContent, tenth * i, expected, 0, tenth);
@@ -139,8 +246,6 @@ public class TestHFlush {
 
       assertEquals("File size doesn't match ", AppendTestUtil.FILE_SIZE, fileSystem.getFileStatus(path).getLen());
       AppendTestUtil.checkFullFile(fileSystem, path, fileContent.length, fileContent, "hflush()");
-    } catch (Exception e) {
-      e.printStackTrace();
     } finally {
       fileSystem.close();
       cluster.shutdown();

+ 96 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java

@@ -30,7 +30,9 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
@@ -49,6 +51,10 @@ public class TestLease {
         ).getLeaseByPath(src.toString()) != null;
   }
 
+  static int leaseCount(MiniDFSCluster cluster) {
+    return NameNodeAdapter.getLeaseManager(cluster.getNamesystem()).countLease();
+  }
+  
   static final String dirString = "/test/lease";
   final Path dir = new Path(dirString);
   static final Log LOG = LogFactory.getLog(TestLease.class);
@@ -126,6 +132,96 @@ public class TestLease {
     }
   }
 
+  @Test
+  public void testLeaseAfterRename() throws Exception {
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
+    try {
+      Path p = new Path("/test-file");
+      Path d = new Path("/test-d");
+      Path d2 = new Path("/test-d-other");
+
+      // open a file to get a lease
+      FileSystem fs = cluster.getFileSystem();
+      FSDataOutputStream out = fs.create(p);
+      out.writeBytes("something");
+      //out.hsync();
+      Assert.assertTrue(hasLease(cluster, p));
+      Assert.assertEquals(1, leaseCount(cluster));
+      
+      // just to ensure first fs doesn't have any logic to twiddle leases
+      DistributedFileSystem fs2 = (DistributedFileSystem) FileSystem.newInstance(fs.getUri(), fs.getConf());
+
+      // rename the file into an existing dir
+      LOG.info("DMS: rename file into dir");
+      Path pRenamed = new Path(d, p.getName());
+      fs2.mkdirs(d);
+      fs2.rename(p, pRenamed);
+      Assert.assertFalse(p+" exists", fs2.exists(p));
+      Assert.assertTrue(pRenamed+" not found", fs2.exists(pRenamed));
+      Assert.assertFalse("has lease for "+p, hasLease(cluster, p));
+      Assert.assertTrue("no lease for "+pRenamed, hasLease(cluster, pRenamed));
+      Assert.assertEquals(1, leaseCount(cluster));
+    
+      // rename the parent dir to a new non-existent dir
+      LOG.info("DMS: rename parent dir");
+      Path pRenamedAgain = new Path(d2, pRenamed.getName());
+      fs2.rename(d, d2);
+      // src gone
+      Assert.assertFalse(d+" exists", fs2.exists(d));
+      Assert.assertFalse("has lease for "+pRenamed, hasLease(cluster, pRenamed));
+      // dst checks
+      Assert.assertTrue(d2+" not found", fs2.exists(d2));
+      Assert.assertTrue(pRenamedAgain+" not found", fs2.exists(pRenamedAgain));
+      Assert.assertTrue("no lease for "+pRenamedAgain, hasLease(cluster, pRenamedAgain));
+      Assert.assertEquals(1, leaseCount(cluster));
+
+      // rename the parent dir to existing dir
+      // NOTE: rename w/o options moves paths into existing dir
+      LOG.info("DMS: rename parent again");
+      pRenamed = pRenamedAgain;
+      pRenamedAgain = new Path(new Path(d, d2.getName()), p.getName());      
+      fs2.mkdirs(d);
+      fs2.rename(d2, d);
+      // src gone
+      Assert.assertFalse(d2+" exists", fs2.exists(d2));
+      Assert.assertFalse("no lease for "+pRenamed, hasLease(cluster, pRenamed));
+      // dst checks
+      Assert.assertTrue(d+" not found", fs2.exists(d));
+      Assert.assertTrue(pRenamedAgain +" not found", fs2.exists(pRenamedAgain));
+      Assert.assertTrue("no lease for "+pRenamedAgain, hasLease(cluster, pRenamedAgain));
+      Assert.assertEquals(1, leaseCount(cluster));
+      
+      // rename with opts to non-existent dir
+      pRenamed = pRenamedAgain;
+      pRenamedAgain = new Path(d2, p.getName());
+      fs2.rename(pRenamed.getParent(), d2, Options.Rename.OVERWRITE);
+      // src gone
+      Assert.assertFalse(pRenamed.getParent() +" not found", fs2.exists(pRenamed.getParent()));
+      Assert.assertFalse("has lease for "+pRenamed, hasLease(cluster, pRenamed));
+      // dst checks
+      Assert.assertTrue(d2+" not found", fs2.exists(d2));
+      Assert.assertTrue(pRenamedAgain+" not found", fs2.exists(pRenamedAgain));
+      Assert.assertTrue("no lease for "+pRenamedAgain, hasLease(cluster, pRenamedAgain));
+      Assert.assertEquals(1, leaseCount(cluster));
+
+      // rename with opts to existing dir
+      // NOTE: rename with options will not move paths into the existing dir
+      pRenamed = pRenamedAgain;
+      pRenamedAgain = new Path(d, p.getName());
+      fs2.rename(pRenamed.getParent(), d, Options.Rename.OVERWRITE);
+      // src gone
+      Assert.assertFalse(pRenamed.getParent() +" not found", fs2.exists(pRenamed.getParent()));
+      Assert.assertFalse("has lease for "+pRenamed, hasLease(cluster, pRenamed));
+      // dst checks
+      Assert.assertTrue(d+" not found", fs2.exists(d));
+      Assert.assertTrue(pRenamedAgain+" not found", fs2.exists(pRenamedAgain));
+      Assert.assertTrue("no lease for "+pRenamedAgain, hasLease(cluster, pRenamedAgain));
+      Assert.assertEquals(1, leaseCount(cluster));
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  
   @Test
   public void testLease() throws Exception {
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();

+ 292 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithNodeGroup.java

@@ -0,0 +1,292 @@
+/**
+ * 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.balancer;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSClusterWithNodeGroup;
+import org.apache.hadoop.hdfs.NameNodeProxies;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.net.NetworkTopology;
+import org.junit.Test;
+
+/**
+ * This class tests if a balancer schedules tasks correctly.
+ */
+public class TestBalancerWithNodeGroup {
+  private static final Log LOG = LogFactory.getLog(
+  "org.apache.hadoop.hdfs.TestBalancerWithNodeGroup");
+  
+  final private static long CAPACITY = 500L;
+  final private static String RACK0 = "/rack0";
+  final private static String RACK1 = "/rack1";
+  final private static String NODEGROUP0 = "/nodegroup0";
+  final private static String NODEGROUP1 = "/nodegroup1";
+  final private static String NODEGROUP2 = "/nodegroup2";
+  final static private String fileName = "/tmp.txt";
+  final static private Path filePath = new Path(fileName);
+  MiniDFSClusterWithNodeGroup cluster;
+
+  ClientProtocol client;
+
+  static final long TIMEOUT = 20000L; //msec
+  static final double CAPACITY_ALLOWED_VARIANCE = 0.005;  // 0.5%
+  static final double BALANCE_ALLOWED_VARIANCE = 0.11;    // 10%+delta
+  static final int DEFAULT_BLOCK_SIZE = 10;
+
+  static {
+    Balancer.setBlockMoveWaitTime(1000L) ;
+  }
+
+  static Configuration createConf() {
+    Configuration conf = new HdfsConfiguration();
+    TestBalancer.initConf(conf);
+    conf.set(CommonConfigurationKeysPublic.NET_TOPOLOGY_IMPL_KEY, 
+        "org.apache.hadoop.net.NetworkTopologyWithNodeGroup");
+    conf.set("dfs.block.replicator.classname", 
+        "org.apache.hadoop.hdfs.server.blockmanagement." +
+        "BlockPlacementPolicyWithNodeGroup");
+    return conf;
+  }
+
+  /**
+   * Wait until heartbeat gives expected results, within CAPACITY_ALLOWED_VARIANCE, 
+   * summed over all nodes.  Times out after TIMEOUT msec.
+   * @param expectedUsedSpace
+   * @param expectedTotalSpace
+   * @throws IOException - if getStats() fails
+   * @throws TimeoutException
+   */
+  private void waitForHeartBeat(long expectedUsedSpace, long expectedTotalSpace)
+  throws IOException, TimeoutException {
+    long timeout = TIMEOUT;
+    long failtime = (timeout <= 0L) ? Long.MAX_VALUE
+             : System.currentTimeMillis() + timeout;
+
+    while (true) {
+      long[] status = client.getStats();
+      double totalSpaceVariance = Math.abs((double)status[0] - expectedTotalSpace) 
+          / expectedTotalSpace;
+      double usedSpaceVariance = Math.abs((double)status[1] - expectedUsedSpace) 
+          / expectedUsedSpace;
+      if (totalSpaceVariance < CAPACITY_ALLOWED_VARIANCE 
+          && usedSpaceVariance < CAPACITY_ALLOWED_VARIANCE)
+        break; //done
+
+      if (System.currentTimeMillis() > failtime) {
+        throw new TimeoutException("Cluster failed to reached expected values of "
+            + "totalSpace (current: " + status[0] 
+            + ", expected: " + expectedTotalSpace 
+            + "), or usedSpace (current: " + status[1] 
+            + ", expected: " + expectedUsedSpace
+            + "), in more than " + timeout + " msec.");
+      }
+      try {
+        Thread.sleep(100L);
+      } catch(InterruptedException ignored) {
+      }
+    }
+  }
+
+  /**
+   * Wait until balanced: each datanode gives utilization within 
+   * BALANCE_ALLOWED_VARIANCE of average
+   * @throws IOException
+   * @throws TimeoutException
+   */
+  private void waitForBalancer(long totalUsedSpace, long totalCapacity) 
+  throws IOException, TimeoutException {
+    long timeout = TIMEOUT;
+    long failtime = (timeout <= 0L) ? Long.MAX_VALUE
+        : System.currentTimeMillis() + timeout;
+    final double avgUtilization = ((double)totalUsedSpace) / totalCapacity;
+    boolean balanced;
+    do {
+      DatanodeInfo[] datanodeReport = 
+          client.getDatanodeReport(DatanodeReportType.ALL);
+      assertEquals(datanodeReport.length, cluster.getDataNodes().size());
+      balanced = true;
+      for (DatanodeInfo datanode : datanodeReport) {
+        double nodeUtilization = ((double)datanode.getDfsUsed())
+            / datanode.getCapacity();
+        if (Math.abs(avgUtilization - nodeUtilization) >
+            BALANCE_ALLOWED_VARIANCE) {
+          balanced = false;
+          if (System.currentTimeMillis() > failtime) {
+            throw new TimeoutException(
+                "Rebalancing expected avg utilization to become "
+                + avgUtilization + ", but on datanode " + datanode
+                + " it remains at " + nodeUtilization
+                + " after more than " + TIMEOUT + " msec.");
+          }
+          try {
+            Thread.sleep(100);
+          } catch (InterruptedException ignored) {
+          }
+          break;
+        }
+      }
+    } while (!balanced);
+  }
+
+  private void runBalancer(Configuration conf,
+      long totalUsedSpace, long totalCapacity) throws Exception {
+    waitForHeartBeat(totalUsedSpace, totalCapacity);
+
+    // start rebalancing
+    Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
+    final int r = Balancer.run(namenodes, Balancer.Parameters.DEFALUT, conf);
+    assertEquals(Balancer.ReturnStatus.SUCCESS.code, r);
+
+    waitForHeartBeat(totalUsedSpace, totalCapacity);
+    LOG.info("Rebalancing with default factor.");
+    waitForBalancer(totalUsedSpace, totalCapacity);
+  }
+
+  /**
+   * Create a cluster with even distribution, and a new empty node is added to
+   * the cluster, then test rack locality for balancer policy. 
+   */
+  @Test
+  public void testBalancerWithRackLocality() throws Exception {
+    Configuration conf = createConf();
+    long[] capacities = new long[]{CAPACITY, CAPACITY};
+    String[] racks = new String[]{RACK0, RACK1};
+    String[] nodeGroups = new String[]{NODEGROUP0, NODEGROUP1};
+    
+    int numOfDatanodes = capacities.length;
+    assertEquals(numOfDatanodes, racks.length);
+    MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf)
+                                .numDataNodes(capacities.length)
+                                .racks(racks)
+                                .simulatedCapacities(capacities);
+    MiniDFSClusterWithNodeGroup.setNodeGroups(nodeGroups);
+    cluster = new MiniDFSClusterWithNodeGroup(builder);
+    try {
+      cluster.waitActive();
+      client = NameNodeProxies.createProxy(conf, 
+          cluster.getFileSystem(0).getUri(),
+          ClientProtocol.class).getProxy();
+
+      long totalCapacity = TestBalancer.sum(capacities);
+
+      // fill up the cluster to be 30% full
+      long totalUsedSpace = totalCapacity * 3 / 10;
+      TestBalancer.createFile(cluster, filePath, totalUsedSpace / numOfDatanodes,
+          (short) numOfDatanodes, 0);
+      
+      long newCapacity = CAPACITY;
+      String newRack = RACK1;
+      String newNodeGroup = NODEGROUP2;
+      // start up an empty node with the same capacity and on the same rack
+      cluster.startDataNodes(conf, 1, true, null, new String[]{newRack},
+          new long[] {newCapacity}, new String[]{newNodeGroup});
+
+      totalCapacity += newCapacity;
+
+      // run balancer and validate results
+      runBalancer(conf, totalUsedSpace, totalCapacity);
+      
+      DatanodeInfo[] datanodeReport = 
+              client.getDatanodeReport(DatanodeReportType.ALL);
+      
+      Map<String, Integer> rackToUsedCapacity = new HashMap<String, Integer>();
+      for (DatanodeInfo datanode: datanodeReport) {
+        String rack = NetworkTopology.getFirstHalf(datanode.getNetworkLocation());
+        int usedCapacity = (int) datanode.getDfsUsed();
+         
+        if (rackToUsedCapacity.get(rack) != null) {
+          rackToUsedCapacity.put(rack, usedCapacity + rackToUsedCapacity.get(rack));
+        } else {
+          rackToUsedCapacity.put(rack, usedCapacity);
+        }
+      }
+      assertEquals(rackToUsedCapacity.size(), 2);
+      assertEquals(rackToUsedCapacity.get(RACK0), rackToUsedCapacity.get(RACK1));
+      
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  
+  /**
+   * Create a cluster with even distribution, and a new empty node is added to
+   * the cluster, then test node-group locality for balancer policy.
+   */
+  @Test
+  public void testBalancerWithNodeGroup() throws Exception {
+    Configuration conf = createConf();
+    long[] capacities = new long[]{CAPACITY, CAPACITY, CAPACITY, CAPACITY};
+    String[] racks = new String[]{RACK0, RACK0, RACK1, RACK1};
+    String[] nodeGroups = new String[]{NODEGROUP0, NODEGROUP0, NODEGROUP1, NODEGROUP2};
+    
+    int numOfDatanodes = capacities.length;
+    assertEquals(numOfDatanodes, racks.length);
+    assertEquals(numOfDatanodes, nodeGroups.length);
+    MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf)
+                                .numDataNodes(capacities.length)
+                                .racks(racks)
+                                .simulatedCapacities(capacities);
+    MiniDFSClusterWithNodeGroup.setNodeGroups(nodeGroups);
+    cluster = new MiniDFSClusterWithNodeGroup(builder);
+    try {
+      cluster.waitActive();
+      client = NameNodeProxies.createProxy(conf, 
+          cluster.getFileSystem(0).getUri(),
+          ClientProtocol.class).getProxy();
+
+      long totalCapacity = TestBalancer.sum(capacities);
+      // fill up the cluster to be 20% full
+      long totalUsedSpace = totalCapacity * 2 / 10;
+      TestBalancer.createFile(cluster, filePath, totalUsedSpace / (numOfDatanodes/2),
+          (short) (numOfDatanodes/2), 0);
+      
+      long newCapacity = CAPACITY;
+      String newRack = RACK1;
+      String newNodeGroup = NODEGROUP2;
+      // start up an empty node with the same capacity and on NODEGROUP2
+      cluster.startDataNodes(conf, 1, true, null, new String[]{newRack},
+          new long[] {newCapacity}, new String[]{newNodeGroup});
+
+      totalCapacity += newCapacity;
+
+      // run balancer and validate results
+      runBalancer(conf, totalUsedSpace, totalCapacity);
+
+    } finally {
+      cluster.shutdown();
+    }
+  }
+}

+ 173 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java

@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -45,6 +46,8 @@ import org.junit.Test;
 public class TestReplicationPolicyWithNodeGroup {
   private static final int BLOCK_SIZE = 1024;
   private static final int NUM_OF_DATANODES = 8;
+  private static final int NUM_OF_DATANODES_BOUNDARY = 6;
+  private static final int NUM_OF_DATANODES_MORE_TARGETS = 12;
   private static final Configuration CONF = new HdfsConfiguration();
   private static final NetworkTopology cluster;
   private static final NameNode namenode;
@@ -61,6 +64,32 @@ public class TestReplicationPolicyWithNodeGroup {
       DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/d2/r3/n5"),
       DFSTestUtil.getDatanodeDescriptor("8.8.8.8", "/d2/r3/n6")
   };
+  
+  private final static DatanodeDescriptor dataNodesInBoundaryCase[] = 
+          new DatanodeDescriptor[] {
+      DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/d1/r1/n1"),
+      DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/d1/r1/n1"),
+      DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/d1/r1/n1"),
+      DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/d1/r1/n2"),
+      DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/d1/r2/n3"),
+      DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/d1/r2/n3")
+  };
+  
+  private final static DatanodeDescriptor dataNodesInMoreTargetsCase[] =
+          new DatanodeDescriptor[] {
+      DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/r1/n1"),
+      DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/r1/n1"),
+      DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/r1/n2"),
+      DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/r1/n2"),
+      DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/r1/n3"),
+      DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/r1/n3"),
+      DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/r2/n4"),
+      DFSTestUtil.getDatanodeDescriptor("8.8.8.8", "/r2/n4"),
+      DFSTestUtil.getDatanodeDescriptor("9.9.9.9", "/r2/n5"),
+      DFSTestUtil.getDatanodeDescriptor("10.10.10.10", "/r2/n5"),
+      DFSTestUtil.getDatanodeDescriptor("11.11.11.11", "/r2/n6"),
+      DFSTestUtil.getDatanodeDescriptor("12.12.12.12", "/r2/n6"),
+  };
 
   private final static DatanodeDescriptor NODE = 
       new DatanodeDescriptor(DFSTestUtil.getDatanodeDescriptor("9.9.9.9", "/d2/r4/n7"));
@@ -74,6 +103,12 @@ public class TestReplicationPolicyWithNodeGroup {
           "org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyWithNodeGroup");
       CONF.set(CommonConfigurationKeysPublic.NET_TOPOLOGY_IMPL_KEY, 
           "org.apache.hadoop.net.NetworkTopologyWithNodeGroup");
+      
+      File baseDir = new File(System.getProperty(
+          "test.build.data", "build/test/data"), "dfs/");
+      CONF.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+          new File(baseDir, "name").getPath());
+      
       DFSTestUtil.formatNameNode(CONF);
       namenode = new NameNode(CONF);
     } catch (IOException e) {
@@ -97,7 +132,27 @@ public class TestReplicationPolicyWithNodeGroup {
           2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
     }
   }
-
+  
+  /**
+   * Scan the targets list: all targets should be on different NodeGroups.
+   * Return false if two targets are found on the same NodeGroup.
+   */
+  private static boolean checkTargetsOnDifferentNodeGroup(
+      DatanodeDescriptor[] targets) {
+    if(targets.length == 0)
+      return true;
+    Set<String> targetSet = new HashSet<String>();
+    for(DatanodeDescriptor node:targets) {
+      String nodeGroup = NetworkTopology.getLastHalf(node.getNetworkLocation());
+      if(targetSet.contains(nodeGroup)) {
+        return false;
+      } else {
+        targetSet.add(nodeGroup);
+      }
+    }
+    return true;
+  }
+  
   /**
    * In this testcase, client is dataNodes[0]. So the 1st replica should be
    * placed on dataNodes[0], the 2nd replica should be placed on 
@@ -497,5 +552,122 @@ public class TestReplicationPolicyWithNodeGroup {
         null, null, (short)1, first, second);
     assertEquals(chosenNode, dataNodes[5]);
   }
+  
+  /**
+   * Test replica placement policy in case of boundary topology.
+   * Rack 2 has only 1 node group & can't be placed with two replicas
+   * The 1st replica will be placed on writer.
+   * The 2nd replica should be placed on a different rack 
+   * The 3rd replica should be placed on the same rack with writer, but on a 
+   * different node group.
+   */
+  @Test
+  public void testChooseTargetsOnBoundaryTopology() throws Exception {
+    for(int i=0; i<NUM_OF_DATANODES; i++) {
+      cluster.remove(dataNodes[i]);
+    }
+
+    for(int i=0; i<NUM_OF_DATANODES_BOUNDARY; i++) {
+      cluster.add(dataNodesInBoundaryCase[i]);
+    }
+    for(int i=0; i<NUM_OF_DATANODES_BOUNDARY; i++) {
+        dataNodes[0].updateHeartbeat(
+                2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+                (HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0);
+        
+      dataNodesInBoundaryCase[i].updateHeartbeat(
+          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
+    }
+
+    DatanodeDescriptor[] targets;
+    targets = replicator.chooseTarget(filename, 0, dataNodesInBoundaryCase[0],
+        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    assertEquals(targets.length, 0);
+    
+    targets = replicator.chooseTarget(filename, 1, dataNodesInBoundaryCase[0],
+        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    assertEquals(targets.length, 1);
+
+    targets = replicator.chooseTarget(filename, 2, dataNodesInBoundaryCase[0],
+        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    assertEquals(targets.length, 2);
+    assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
+    
+    targets = replicator.chooseTarget(filename, 3, dataNodesInBoundaryCase[0],
+        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    assertEquals(targets.length, 3);
+    assertTrue(checkTargetsOnDifferentNodeGroup(targets));
+  }
+
+  /**
+   * Test re-replication policy in boundary case.
+   * Rack 2 has only one node group & the node in this node group is chosen
+   * Rack 1 has two nodegroups & one of them is chosen.
+   * Replica policy should choose the node from node group of Rack1 but not the
+   * same nodegroup with chosen nodes.
+   */
+  @Test
+  public void testRereplicateOnBoundaryTopology() throws Exception {
+    for(int i=0; i<NUM_OF_DATANODES_BOUNDARY; i++) {
+      dataNodesInBoundaryCase[i].updateHeartbeat(
+          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
+    }
+    List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
+    chosenNodes.add(dataNodesInBoundaryCase[0]);
+    chosenNodes.add(dataNodesInBoundaryCase[5]);
+    DatanodeDescriptor[] targets;
+    targets = replicator.chooseTarget(filename, 1, dataNodesInBoundaryCase[0],
+        chosenNodes, BLOCK_SIZE);
+    assertFalse(cluster.isOnSameNodeGroup(targets[0], 
+        dataNodesInBoundaryCase[0]));
+    assertFalse(cluster.isOnSameNodeGroup(targets[0],
+        dataNodesInBoundaryCase[5]));
+    assertTrue(checkTargetsOnDifferentNodeGroup(targets));
+  }
+  
+  /**
+   * Test replica placement policy in case of targets more than number of 
+   * NodeGroups.
+   * The 12-nodes cluster only has 6 NodeGroups, but in some cases, like: 
+   * placing submitted job file, there is requirement to choose more (10) 
+   * targets for placing replica. We should test it can return 6 targets.
+   */
+  @Test
+  public void testChooseMoreTargetsThanNodeGroups() throws Exception {
+    // Cleanup nodes in previous tests
+    for(int i=0; i<NUM_OF_DATANODES_BOUNDARY; i++) {
+      DatanodeDescriptor node = dataNodesInBoundaryCase[i];
+      if (cluster.contains(node)) {
+        cluster.remove(node);
+      }
+    }
+
+    for(int i=0; i<NUM_OF_DATANODES_MORE_TARGETS; i++) {
+      cluster.add(dataNodesInMoreTargetsCase[i]);
+    }
+
+    for(int i=0; i<NUM_OF_DATANODES_MORE_TARGETS; i++) {
+      dataNodesInMoreTargetsCase[i].updateHeartbeat(
+          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+          2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
+    }
+
+    DatanodeDescriptor[] targets;
+    // Test normal case -- 3 replicas
+    targets = replicator.chooseTarget(filename, 3, dataNodesInMoreTargetsCase[0],
+        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    assertEquals(targets.length, 3);
+    assertTrue(checkTargetsOnDifferentNodeGroup(targets));
+
+    // Test special case -- replica number over node groups.
+    targets = replicator.chooseTarget(filename, 10, dataNodesInMoreTargetsCase[0],
+        new ArrayList<DatanodeDescriptor>(), BLOCK_SIZE);
+    assertTrue(checkTargetsOnDifferentNodeGroup(targets));
+    // Verify it only can find 6 targets for placing replicas.
+    assertEquals(targets.length, 6);
+  }
+
 
 }

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java

@@ -29,6 +29,7 @@ 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.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -41,7 +42,6 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
-import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat.State;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
@@ -123,7 +123,7 @@ public class TestBPOfferService {
           Mockito.anyInt(),
           Mockito.anyInt(),
           Mockito.anyInt());
-    mockHaStatuses[nnIdx] = new NNHAStatusHeartbeat(State.STANDBY, 0);
+    mockHaStatuses[nnIdx] = new NNHAStatusHeartbeat(HAServiceState.STANDBY, 0);
     return mock;
   }
   
@@ -255,12 +255,12 @@ public class TestBPOfferService {
       assertNull(bpos.getActiveNN());
 
       // Have NN1 claim active at txid 1
-      mockHaStatuses[0] = new NNHAStatusHeartbeat(State.ACTIVE, 1);
+      mockHaStatuses[0] = new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 1);
       bpos.triggerHeartbeatForTests();
       assertSame(mockNN1, bpos.getActiveNN());
 
       // NN2 claims active at a higher txid
-      mockHaStatuses[1] = new NNHAStatusHeartbeat(State.ACTIVE, 2);
+      mockHaStatuses[1] = new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 2);
       bpos.triggerHeartbeatForTests();
       assertSame(mockNN2, bpos.getActiveNN());
       
@@ -272,12 +272,12 @@ public class TestBPOfferService {
       // Even if NN2 goes to standby, DN shouldn't reset to talking to NN1,
       // because NN1's txid is lower than the last active txid. Instead,
       // it should consider neither active.
-      mockHaStatuses[1] = new NNHAStatusHeartbeat(State.STANDBY, 2);
+      mockHaStatuses[1] = new NNHAStatusHeartbeat(HAServiceState.STANDBY, 2);
       bpos.triggerHeartbeatForTests();
       assertNull(bpos.getActiveNN());
       
       // Now if NN1 goes back to a higher txid, it should be considered active
-      mockHaStatuses[0] = new NNHAStatusHeartbeat(State.ACTIVE, 3);
+      mockHaStatuses[0] = new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 3);
       bpos.triggerHeartbeatForTests();
       assertSame(mockNN1, bpos.getActiveNN());
 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java

@@ -49,6 +49,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -72,7 +73,6 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
-import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat.State;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
@@ -157,7 +157,7 @@ public class TestBlockRecovery {
             Mockito.anyInt()))
         .thenReturn(new HeartbeatResponse(
             new DatanodeCommand[0],
-            new NNHAStatusHeartbeat(State.ACTIVE, 1)));
+            new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 1)));
 
     dn = new DataNode(conf, dirs, null) {
       @Override

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

@@ -179,6 +179,13 @@ public class NameNodeAdapter {
     return spy;
   }
   
+  public static JournalSet spyOnJournalSet(NameNode nn) {
+    FSEditLog editLog = nn.getFSImage().getEditLog();
+    JournalSet js = Mockito.spy(editLog.getJournalSet());
+    editLog.setJournalSetForTesting(js);
+    return js;
+  }
+  
   public static String getMkdirOpPath(FSEditLogOp op) {
     if (op.opCode == FSEditLogOpCodes.OP_MKDIR) {
       return ((MkdirOp) op).path;

+ 123 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java

@@ -0,0 +1,123 @@
+/**
+ * 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 static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.net.InetAddress;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.Test;
+
+/**
+ * Tests for the {@link AuditLogger} custom audit logging interface.
+ */
+public class TestAuditLogger {
+
+  /**
+   * Tests that AuditLogger works as expected.
+   */
+  @Test
+  public void testAuditLogger() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    conf.set(DFS_NAMENODE_AUDIT_LOGGERS_KEY,
+        DummyAuditLogger.class.getName());
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+
+    try {
+      cluster.waitClusterUp();
+      assertTrue(DummyAuditLogger.initialized);
+
+      FileSystem fs = cluster.getFileSystem();
+      long time = System.currentTimeMillis();
+      fs.setTimes(new Path("/"), time, time);
+      assertEquals(1, DummyAuditLogger.logCount);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Tests that a broken audit logger causes requests to fail.
+   */
+  @Test
+  public void testBrokenLogger() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    conf.set(DFS_NAMENODE_AUDIT_LOGGERS_KEY,
+        BrokenAuditLogger.class.getName());
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+
+    try {
+      cluster.waitClusterUp();
+
+      FileSystem fs = cluster.getFileSystem();
+      long time = System.currentTimeMillis();
+      fs.setTimes(new Path("/"), time, time);
+      fail("Expected exception due to broken audit logger.");
+    } catch (RemoteException re) {
+      // Expected.
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  public static class DummyAuditLogger implements AuditLogger {
+
+    static boolean initialized;
+    static int logCount;
+
+    public void initialize(Configuration conf) {
+      initialized = true;
+    }
+
+    public void logAuditEvent(boolean succeeded, String userName,
+        InetAddress addr, String cmd, String src, String dst,
+        FileStatus stat) {
+      logCount++;
+    }
+
+  }
+
+  public static class BrokenAuditLogger implements AuditLogger {
+
+    public void initialize(Configuration conf) {
+      // No op.
+    }
+
+    public void logAuditEvent(boolean succeeded, String userName,
+        InetAddress addr, String cmd, String src, String dst,
+        FileStatus stat) {
+      throw new RuntimeException("uh oh");
+    }
+
+  }
+
+}

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

@@ -35,6 +35,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HAUtil;
@@ -103,6 +104,9 @@ public class TestBackupNode {
     BackupNode bn = (BackupNode)NameNode.createNameNode(
         new String[]{startupOpt.getName()}, c);
     assertTrue(bn.getRole() + " must be in SafeMode.", bn.isInSafeMode());
+    assertTrue(bn.getRole() + " must be in StandbyState",
+               bn.getNamesystem().getHAState()
+                 .equalsIgnoreCase(HAServiceState.STANDBY.name()));
     return bn;
   }
 

+ 47 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -25,10 +25,15 @@ import static org.junit.Assert.fail;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.junit.Test;
 
@@ -157,6 +162,48 @@ public class TestINodeFile {
     
   }
   
+  /**
+   * FSDirectory#unprotectedSetQuota creates a new INodeDirectoryWithQuota to
+   * replace the original INodeDirectory. Before HDFS-4243, the parent field of
+   * all the children INodes of the target INodeDirectory is not changed to
+   * point to the new INodeDirectoryWithQuota. This testcase tests this
+   * scenario.
+   */
+  @Test
+  public void testGetFullPathNameAfterSetQuota() throws Exception {
+    long fileLen = 1024;
+    replication = 3;
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
+        replication).build();
+    cluster.waitActive();
+    FSNamesystem fsn = cluster.getNamesystem();
+    FSDirectory fsdir = fsn.getFSDirectory();
+    DistributedFileSystem dfs = cluster.getFileSystem();
+    
+    // Create a file for test
+    final Path dir = new Path("/dir");
+    final Path file = new Path(dir, "file");
+    DFSTestUtil.createFile(dfs, file, fileLen, replication, 0L);
+    
+    // Check the full path name of the INode associating with the file
+    INode fnode = fsdir.getINode(file.toString());
+    assertEquals(file.toString(), fnode.getFullPathName());
+    
+    // Call FSDirectory#unprotectedSetQuota which calls
+    // INodeDirectory#replaceChild
+    dfs.setQuota(dir, Long.MAX_VALUE - 1, replication * fileLen * 10);
+    final Path newDir = new Path("/newdir");
+    final Path newFile = new Path(newDir, "file");
+    // Also rename dir
+    dfs.rename(dir, newDir, Options.Rename.OVERWRITE);
+    // /dir/file now should be renamed to /newdir/file
+    fnode = fsdir.getINode(newFile.toString());
+    // getFullPathName can return correct result only if the parent field of
+    // child node is set correctly
+    assertEquals(newFile.toString(), fnode.getFullPathName());
+  }
+  
   @Test
   public void testAppendBlocks() {
     INodeFile origFile = createINodeFiles(1, "origfile")[0];

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java

@@ -92,7 +92,8 @@ public class TestMetaSave {
     DataInputStream in = new DataInputStream(fstream);
     BufferedReader reader = new BufferedReader(new InputStreamReader(in));
     String line = reader.readLine();
-    assertTrue(line.equals("3 files and directories, 2 blocks = 5 total"));
+    assertTrue(line.equals(
+      "3 files and directories, 2 blocks = 5 total filesystem objects"));
     line = reader.readLine();
     assertTrue(line.equals("Live Datanodes: 1"));
     line = reader.readLine();

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java

@@ -603,6 +603,24 @@ public class TestSaveNamespace {
     }
   }
   
+  @Test
+  public void testSaveNamespaceWithDanglingLease() throws Exception {
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(new Configuration())
+        .numDataNodes(1).build();
+    cluster.waitActive();
+    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    try {
+      cluster.getNamesystem().leaseManager.addLease("me", "/non-existent");      
+      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      cluster.getNameNodeRpc().saveNamespace();
+      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
   private void doAnEdit(FSNamesystem fsn, int id) throws IOException {
     // Make an edit
     fsn.mkdirs(

+ 17 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.JournalSet;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
@@ -66,6 +67,12 @@ public class TestStandbyCheckpoints {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 5);
     conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+    
+    // Dial down the retention of extra edits and checkpoints. This is to
+    // help catch regressions of HDFS-4238 (SBN should not purge shared edits)
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_KEY, 0);
+    
     conf.setBoolean(DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, true);
     conf.set(DFSConfigKeys.DFS_IMAGE_COMPRESSION_CODEC_KEY,
         SlowCodec.class.getCanonicalName());
@@ -99,15 +106,20 @@ public class TestStandbyCheckpoints {
 
   @Test
   public void testSBNCheckpoints() throws Exception {
-    doEdits(0, 10);
+    JournalSet standbyJournalSet = NameNodeAdapter.spyOnJournalSet(nn1);
     
+    doEdits(0, 10);
     HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
     // Once the standby catches up, it should notice that it needs to
     // do a checkpoint and save one to its local directories.
-    HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(0, 12));
+    HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(12));
     
     // It should also upload it back to the active.
-    HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(0, 12));
+    HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(12));
+    
+    // The standby should never try to purge edit logs on shared storage.
+    Mockito.verify(standbyJournalSet, Mockito.never()).
+      purgeLogsOlderThan(Mockito.anyLong());
   }
 
   /**
@@ -129,8 +141,8 @@ public class TestStandbyCheckpoints {
     // so the standby will catch up. Then, both will be in standby mode
     // with enough uncheckpointed txns to cause a checkpoint, and they
     // will each try to take a checkpoint and upload to each other.
-    HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(0, 12));
-    HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(0, 12));
+    HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(12));
+    HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(12));
     
     assertEquals(12, nn0.getNamesystem().getFSImage()
         .getMostRecentCheckpointTxId());

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

@@ -605,6 +605,15 @@ Release 0.23.6 - UNRELEASED
 
     MAPREDUCE-4764. repair TestBinaryTokenFile (Ivan A. Veselovsky via bobby)
 
+    MAPREDUCE-4825. JobImpl.finished doesn't expect ERROR as a final job state
+    (jlowe via bobby)
+
+    MAPREDUCE-4817. Hardcoded task ping timeout kills tasks localizing large 
+    amounts of data (tgraves)
+
+    MAPREDUCE-4836. Elapsed time for running tasks on AM web UI tasks page is 0
+    (Ravi Prakash via jeagles)
+
 Release 0.23.5 - UNRELEASED
 
   INCOMPATIBLE CHANGES

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

@@ -274,7 +274,6 @@ public class TaskAttemptListenerImpl extends CompositeService
   @Override
   public boolean ping(TaskAttemptID taskAttemptID) throws IOException {
     LOG.info("Ping from " + taskAttemptID.toString());
-    taskHeartbeatHandler.pinged(TypeConverter.toYarn(taskAttemptID));
     return true;
   }
 

+ 3 - 24
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java

@@ -46,33 +46,22 @@ import org.apache.hadoop.yarn.service.AbstractService;
 public class TaskHeartbeatHandler extends AbstractService {
   
   private static class ReportTime {
-    private long lastPing;
     private long lastProgress;
     
     public ReportTime(long time) {
       setLastProgress(time);
     }
     
-    public synchronized void setLastPing(long time) {
-      lastPing = time;
-    }
-    
     public synchronized void setLastProgress(long time) {
       lastProgress = time;
-      lastPing = time;
-    }
-    
-    public synchronized long getLastPing() {
-      return lastPing;
     }
-    
+
     public synchronized long getLastProgress() {
       return lastProgress;
     }
   }
   
   private static final Log LOG = LogFactory.getLog(TaskHeartbeatHandler.class);
-  private static final int PING_TIMEOUT = 5 * 60 * 1000;
   
   //thread which runs periodically to see the last time since a heartbeat is
   //received from a task.
@@ -127,14 +116,6 @@ public class TaskHeartbeatHandler extends AbstractService {
     }
   }
 
-  public void pinged(TaskAttemptId attemptID) {
-    //only put for the registered attempts
-      //TODO throw an exception if the task isn't registered.
-      ReportTime time = runningAttempts.get(attemptID);
-      if(time != null) {
-        time.setLastPing(clock.getTime());
-      }
-    }
   
   public void register(TaskAttemptId attemptID) {
     runningAttempts.put(attemptID, new ReportTime(clock.getTime()));
@@ -159,10 +140,8 @@ public class TaskHeartbeatHandler extends AbstractService {
           Map.Entry<TaskAttemptId, ReportTime> entry = iterator.next();
           boolean taskTimedOut = (taskTimeOut > 0) && 
               (currentTime > (entry.getValue().getLastProgress() + taskTimeOut));
-          boolean pingTimedOut =
-              (currentTime > (entry.getValue().getLastPing() + PING_TIMEOUT));
-              
-          if(taskTimedOut || pingTimedOut) {
+           
+          if(taskTimedOut) {
             // task is lost, remove from the list and raise lost event
             iterator.remove();
             eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(entry

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

@@ -828,6 +828,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       case KILLED:
         metrics.killedJob(this);
         break;
+      case ERROR:
       case FAILED:
         metrics.failedJob(this);
         break;

+ 3 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskInfo.java

@@ -59,11 +59,12 @@ public class TaskInfo {
     TaskReport report = task.getReport();
     this.startTime = report.getStartTime();
     this.finishTime = report.getFinishTime();
-    this.elapsedTime = Times.elapsed(this.startTime, this.finishTime, false);
+    this.state = report.getTaskState();
+    this.elapsedTime = Times.elapsed(this.startTime, this.finishTime,
+      this.state == TaskState.RUNNING);
     if (this.elapsedTime == -1) {
       this.elapsedTime = 0;
     }
-    this.state = report.getTaskState();
     this.progress = report.getProgress() * 100;
     this.id = MRApps.toString(task.getID());
     this.taskNum = task.getID().getId();

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

@@ -115,6 +115,14 @@ Release 2.0.3-alpha - Unreleased
     MAPREDUCE-4778. Fair scheduler event log is only written if directory
     exists on HDFS. (Sandy Ryza via tomwhite)
 
+    YARN-229. Remove old unused RM recovery code. (Bikas Saha via acmurthy) 
+
+    YARN-187. Add hierarchical queues to the fair scheduler.
+    (Sandy Ryza via tomwhite)
+
+    YARN-72. NM should handle cleaning up containers when it shuts down.
+    (Sandy Ryza via tomwhite)
+
 Release 2.0.2-alpha - 2012-09-07 
 
   INCOMPATIBLE CHANGES
@@ -192,6 +200,12 @@ Release 0.23.6 - UNRELEASED
     YARN-204. test coverage for org.apache.hadoop.tools (Aleksey Gorshkov via
     bobby)
 
+    YARN-251. Proxy URI generation fails for blank tracking URIs (Tom White
+    via jlowe)
+
+    YARN-258. RM web page UI shows Invalid Date for start and finish times
+    (Ravi Prakash via jlowe)
+
 Release 0.23.5 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestGetGroups.java

@@ -28,7 +28,7 @@ import org.apache.hadoop.tools.GetGroupsTestBase;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.StoreFactory;
 import org.apache.hadoop.yarn.service.Service.STATE;
 import org.junit.AfterClass;
@@ -46,7 +46,7 @@ public class TestGetGroups extends GetGroupsTestBase {
   @BeforeClass
   public static void setUpResourceManager() throws IOException, InterruptedException {
     conf = new YarnConfiguration();
-    Store store = StoreFactory.getStore(conf);
+    RMStateStore store = StoreFactory.getStore(conf);
     resourceManager = new ResourceManager(store) {
       @Override
       protected void doSecureLogin() throws IOException {

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

@@ -228,16 +228,7 @@ public class YarnConfiguration extends Configuration {
   
   /** The class to use as the persistent store.*/
   public static final String RM_STORE = RM_PREFIX + "store.class";
- 
-  /** The address of the zookeeper instance to use with ZK store.*/
-  public static final String RM_ZK_STORE_ADDRESS = 
-    RM_PREFIX + "zookeeper-store.address";
-  
-  /** The zookeeper session timeout for the zookeeper store.*/
-  public static final String RM_ZK_STORE_TIMEOUT_MS = 
-    RM_PREFIX + "zookeeper-store.session.timeout-ms";
-  public static final int DEFAULT_RM_ZK_STORE_TIMEOUT_MS = 60000;
-  
+   
   /** The maximum number of completed applications RM keeps. */ 
   public static final String RM_MAX_COMPLETED_APPLICATIONS =
     RM_PREFIX + "max-completed-applications";

+ 0 - 19
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java

@@ -28,17 +28,6 @@ import static org.apache.hadoop.yarn.util.StringHelper.*;
 import org.apache.hadoop.yarn.webapp.hamlet.HamletSpec.HTML;
 
 public class JQueryUI extends HtmlBlock {
-  // Render choices (mostly for dataTables)
-  public enum Render {
-    /** small (<~100 rows) table as html, most gracefully degradable */
-    HTML,
-    /** medium (<~2000 rows) table as js array */
-    JS_ARRAY,
-    /** large (<~10000 rows) table loading from server */
-    JS_LOAD,
-    /** huge (>~10000 rows) table processing from server */
-    JS_SERVER
-  };
 
   // UI params
   public static final String ACCORDION = "ui.accordion";
@@ -197,12 +186,4 @@ public class JQueryUI extends HtmlBlock {
         append("sPaginationType: 'full_numbers', iDisplayLength:20, ").
         append("aLengthMenu:[20, 40, 60, 80, 100]");
   }
-
-  public static StringBuilder tableInitProgress(StringBuilder init,
-                                                long numCells) {
-    return init.append(", bProcessing:true, ").
-        append("oLanguage:{sProcessing:'Processing ").
-        append(numCells).append(" cells...").
-        append("<p><img src=\"/static/busy.gif\">'}");
-  }
 }

+ 0 - 56
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/Jsons.java

@@ -1,56 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.hadoop.yarn.webapp.view;
-
-import java.io.PrintWriter;
-
-import static org.apache.hadoop.yarn.util.StringHelper.*;
-import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
-
-/**
- * JSON helpers
- */
-public class Jsons {
-  public static final String _SEP = "\",\"";
-
-  public static PrintWriter appendProgressBar(PrintWriter out, String pct) {
-    return out.append("<br title='").append(pct).append("'>").
-        append("<div class='").append(C_PROGRESSBAR).
-        append("' title='").append(pct).append('%').
-        append("'><div class='").append(C_PROGRESSBAR_VALUE).
-        append("' style='width: ").append(pct).
-        append("%'>").append("<\\/div><\\/div>");
-  }
-
-  public static PrintWriter appendProgressBar(PrintWriter out,
-                                                  float progress) {
-    return appendProgressBar(out, String.format("%.1f", progress));
-  }
-
-  public static PrintWriter appendSortable(PrintWriter out, Object value) {
-    return out.append("<br title='").append(String.valueOf(value)).append("'>");
-  }
-
-  public static PrintWriter appendLink(PrintWriter out, Object anchor,
-                                       String prefix, String... parts) {
-    String anchorText = String.valueOf(anchor);
-    return out.append("<a href='").append(anchor == null ? "#" :
-      ujoin(prefix, parts)).append("'>").append(anchorText).append("<\\/a>");
-  }
-}

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

@@ -209,17 +209,6 @@
     <name>yarn.resourcemanager.store.class</name>
   </property>
 
-  <property>
-    <description>The address of the zookeeper instance to use with ZK store.</description>
-    <name>yarn.resourcemanager.zookeeper-store.address</name>
-  </property>
-
-  <property>
-    <description>The zookeeper session timeout for the zookeeper store.</description>
-    <name>yarn.resourcemanager.zookeeper-store.session.timeout-ms</name>
-    <value>60000</value>
-  </property>
-
   <property>
     <description>The maximum number of completed applications RM keeps. </description>
     <name>yarn.resourcemanager.max-completed-applications</name>

+ 12 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrCompletedContainersEvent.java

@@ -25,13 +25,23 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 public class CMgrCompletedContainersEvent extends ContainerManagerEvent {
 
   private List<ContainerId> containerToCleanup;
-
-  public CMgrCompletedContainersEvent(List<ContainerId> containersToCleanup) {
+  private Reason reason;
+  
+  public CMgrCompletedContainersEvent(List<ContainerId> containersToCleanup, Reason reason) {
     super(ContainerManagerEventType.FINISH_CONTAINERS);
     this.containerToCleanup = containersToCleanup;
+    this.reason = reason;
   }
 
   public List<ContainerId> getContainersToCleanup() {
     return this.containerToCleanup;
   }
+  
+  public Reason getReason() {
+    return reason;
+  }
+  
+  public static enum Reason {
+    ON_SHUTDOWN, BY_RESOURCEMANAGER
+  }
 }

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

@@ -19,6 +19,9 @@
 package org.apache.hadoop.yarn.server.nodemanager;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentSkipListMap;
@@ -61,14 +64,24 @@ public class NodeManager extends CompositeService implements
    * Priority of the NodeManager shutdown hook.
    */
   public static final int SHUTDOWN_HOOK_PRIORITY = 30;
+  
+  /**
+   * Extra duration to wait for containers to be killed on shutdown.
+   */
+  private static final int SHUTDOWN_CLEANUP_SLOP_MS = 1000;
 
   private static final Log LOG = LogFactory.getLog(NodeManager.class);
   protected final NodeManagerMetrics metrics = NodeManagerMetrics.create();
   private ApplicationACLsManager aclsManager;
   private NodeHealthCheckerService nodeHealthChecker;
   private LocalDirsHandlerService dirsHandler;
+  private Context context;
+  private AsyncDispatcher dispatcher;
+  private ContainerManagerImpl containerManager;
   private static CompositeServiceShutdownHook nodeManagerShutdownHook; 
   
+  private long waitForContainersOnShutdownMillis;
+  
   public NodeManager() {
     super(NodeManager.class.getName());
   }
@@ -115,7 +128,7 @@ public class NodeManager extends CompositeService implements
       containerTokenSecretManager = new NMContainerTokenSecretManager(conf);
     }
 
-    Context context = new NMContext(containerTokenSecretManager);
+    this.context = new NMContext(containerTokenSecretManager);
 
     this.aclsManager = new ApplicationACLsManager(conf);
 
@@ -131,7 +144,7 @@ public class NodeManager extends CompositeService implements
     addService(del);
 
     // NodeManager level dispatcher
-    AsyncDispatcher dispatcher = new AsyncDispatcher();
+    this.dispatcher = new AsyncDispatcher();
 
     nodeHealthChecker = new NodeHealthCheckerService();
     addService(nodeHealthChecker);
@@ -144,7 +157,7 @@ public class NodeManager extends CompositeService implements
     NodeResourceMonitor nodeResourceMonitor = createNodeResourceMonitor();
     addService(nodeResourceMonitor);
 
-    ContainerManagerImpl containerManager =
+    containerManager =
         createContainerManager(context, exec, del, nodeStatusUpdater,
         this.aclsManager, dirsHandler);
     addService(containerManager);
@@ -155,13 +168,20 @@ public class NodeManager extends CompositeService implements
 
     dispatcher.register(ContainerManagerEventType.class, containerManager);
     addService(dispatcher);
-
+    
     DefaultMetricsSystem.initialize("NodeManager");
 
     // StatusUpdater should be added last so that it get started last 
     // so that we make sure everything is up before registering with RM. 
     addService(nodeStatusUpdater);
-
+    
+    waitForContainersOnShutdownMillis =
+        conf.getLong(YarnConfiguration.NM_SLEEP_DELAY_BEFORE_SIGKILL_MS,
+            YarnConfiguration.DEFAULT_NM_SLEEP_DELAY_BEFORE_SIGKILL_MS) + 
+        conf.getLong(YarnConfiguration.NM_PROCESS_KILL_WAIT_MS,
+            YarnConfiguration.DEFAULT_NM_PROCESS_KILL_WAIT_MS) +
+        SHUTDOWN_CLEANUP_SLOP_MS;
+    
     super.init(conf);
     // TODO add local dirs to del
   }
@@ -178,9 +198,44 @@ public class NodeManager extends CompositeService implements
 
   @Override
   public void stop() {
+    cleanupContainers();
     super.stop();
     DefaultMetricsSystem.shutdown();
   }
+  
+  @SuppressWarnings("unchecked")
+  private void cleanupContainers() {
+    Map<ContainerId, Container> containers = context.getContainers();
+    if (containers.isEmpty()) {
+      return;
+    }
+    LOG.info("Containers still running on shutdown: " + containers.keySet());
+    
+    List<ContainerId> containerIds = new ArrayList<ContainerId>(containers.keySet());
+    dispatcher.getEventHandler().handle(
+        new CMgrCompletedContainersEvent(containerIds, 
+            CMgrCompletedContainersEvent.Reason.ON_SHUTDOWN));
+    
+    LOG.info("Waiting for containers to be killed");
+    
+    long waitStartTime = System.currentTimeMillis();
+    while (!containers.isEmpty() && 
+        System.currentTimeMillis() - waitStartTime < waitForContainersOnShutdownMillis) {
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException ex) {
+        LOG.warn("Interrupted while sleeping on container kill", ex);
+      }
+    }
+
+    // All containers killed
+    if (containers.isEmpty()) {
+      LOG.info("All containers in DONE state");
+    } else {
+      LOG.info("Done waiting for containers to be killed. Still alive: " + 
+          containers.keySet());
+    }
+  }
 
   public static class NMContext implements Context {
 
@@ -282,6 +337,11 @@ public class NodeManager extends CompositeService implements
   NodeManager createNewNodeManager() {
     return new NodeManager();
   }
+  
+  // For testing
+  ContainerManagerImpl getContainerManager() {
+    return containerManager;
+  }
 
   public static void main(String[] args) {
     Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());

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

@@ -363,7 +363,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
                 .getContainersToCleanupList();
             if (containersToCleanup.size() != 0) {
               dispatcher.getEventHandler().handle(
-                  new CMgrCompletedContainersEvent(containersToCleanup));
+                  new CMgrCompletedContainersEvent(containersToCleanup, 
+                      CMgrCompletedContainersEvent.Reason.BY_RESOURCEMANAGER));
             }
             List<ApplicationId> appsToCleanup =
                 response.getApplicationsToCleanupList();

+ 11 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java

@@ -23,6 +23,8 @@ import static org.apache.hadoop.yarn.service.Service.STATE.STARTED;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
@@ -593,9 +595,16 @@ public class ContainerManagerImpl extends CompositeService implements
           (CMgrCompletedContainersEvent) event;
       for (ContainerId container : containersFinishedEvent
           .getContainersToCleanup()) {
+        String diagnostic = "";
+        if (containersFinishedEvent.getReason() == 
+            CMgrCompletedContainersEvent.Reason.ON_SHUTDOWN) {
+          diagnostic = "Container Killed on Shutdown";
+        } else if (containersFinishedEvent.getReason() == 
+            CMgrCompletedContainersEvent.Reason.BY_RESOURCEMANAGER) {
+          diagnostic = "Container Killed by ResourceManager";
+        }
         this.dispatcher.getEventHandler().handle(
-            new ContainerKillEvent(container,
-                "Container Killed by ResourceManager"));
+            new ContainerKillEvent(container, diagnostic));
       }
       break;
     default:

+ 92 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/MockNodeStatusUpdater.java

@@ -0,0 +1,92 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.nodemanager;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.server.api.ResourceTracker;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
+import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse;
+import org.apache.hadoop.yarn.server.api.records.NodeStatus;
+import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
+import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
+
+/**
+ * This class allows a node manager to run without without communicating with a
+ * real RM.
+ */
+public class MockNodeStatusUpdater extends NodeStatusUpdaterImpl {
+  static final Log LOG = LogFactory.getLog(MockNodeStatusUpdater.class);
+  
+  private static final RecordFactory recordFactory = RecordFactoryProvider
+      .getRecordFactory(null);
+
+  private ResourceTracker resourceTracker;
+
+  public MockNodeStatusUpdater(Context context, Dispatcher dispatcher,
+      NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
+    super(context, dispatcher, healthChecker, metrics);
+    resourceTracker = new MockResourceTracker();
+  }
+
+  @Override
+  protected ResourceTracker getRMClient() {
+    return resourceTracker;
+  }
+  
+  private static class MockResourceTracker implements ResourceTracker {
+    private int heartBeatID;
+
+    @Override
+    public RegisterNodeManagerResponse registerNodeManager(
+        RegisterNodeManagerRequest request) throws YarnRemoteException {
+      RegistrationResponse regResponse = recordFactory
+          .newRecordInstance(RegistrationResponse.class);
+
+      RegisterNodeManagerResponse response = recordFactory
+          .newRecordInstance(RegisterNodeManagerResponse.class);
+      response.setRegistrationResponse(regResponse);
+      return response;
+    }
+
+    @Override
+    public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
+        throws YarnRemoteException {
+      NodeStatus nodeStatus = request.getNodeStatus();
+      LOG.info("Got heartbeat number " + heartBeatID);
+      nodeStatus.setResponseId(heartBeatID++);
+
+      HeartbeatResponse response = recordFactory
+          .newRecordInstance(HeartbeatResponse.class);
+      response.setResponseId(heartBeatID);
+
+      NodeHeartbeatResponse nhResponse = recordFactory
+          .newRecordInstance(NodeHeartbeatResponse.class);
+      nhResponse.setHeartbeatResponse(response);
+      return nhResponse;
+    }
+  }
+}

+ 222 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java

@@ -0,0 +1,222 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.nodemanager;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.URL;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestNodeManagerShutdown {
+  static final File basedir =
+      new File("target", TestNodeManagerShutdown.class.getName());
+  static final File tmpDir = new File(basedir, "tmpDir");
+  static final File logsDir = new File(basedir, "logs");
+  static final File remoteLogsDir = new File(basedir, "remotelogs");
+  static final File nmLocalDir = new File(basedir, "nm0");
+  static final File processStartFile = new File(tmpDir, "start_file.txt")
+    .getAbsoluteFile();
+
+  static final RecordFactory recordFactory = RecordFactoryProvider
+      .getRecordFactory(null);
+  static final String user = "nobody";
+  private FileContext localFS;
+
+  @Before
+  public void setup() throws UnsupportedFileSystemException {
+    localFS = FileContext.getLocalFSFileContext();
+    tmpDir.mkdirs();
+    logsDir.mkdirs();
+    remoteLogsDir.mkdirs();
+    nmLocalDir.mkdirs();
+  }
+  
+  @After
+  public void tearDown() throws IOException, InterruptedException {
+    localFS.delete(new Path(basedir.getPath()), true);
+  }
+  
+  @Test
+  public void testKillContainersOnShutdown() throws IOException {
+    NodeManager nm = getNodeManager();
+    nm.init(createNMConfig());
+    nm.start();
+    
+    ContainerManagerImpl containerManager = nm.getContainerManager();
+    File scriptFile = createUnhaltingScriptFile();
+    
+    ContainerLaunchContext containerLaunchContext = 
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    // Construct the Container-id
+    ContainerId cId = createContainerId();
+    containerLaunchContext.setContainerId(cId);
+
+    containerLaunchContext.setUser(user);
+
+    URL localResourceUri =
+        ConverterUtils.getYarnUrlFromPath(localFS
+            .makeQualified(new Path(scriptFile.getAbsolutePath())));
+    LocalResource localResource =
+        recordFactory.newRecordInstance(LocalResource.class);
+    localResource.setResource(localResourceUri);
+    localResource.setSize(-1);
+    localResource.setVisibility(LocalResourceVisibility.APPLICATION);
+    localResource.setType(LocalResourceType.FILE);
+    localResource.setTimestamp(scriptFile.lastModified());
+    String destinationFile = "dest_file";
+    Map<String, LocalResource> localResources = 
+        new HashMap<String, LocalResource>();
+    localResources.put(destinationFile, localResource);
+    containerLaunchContext.setLocalResources(localResources);
+    containerLaunchContext.setUser(containerLaunchContext.getUser());
+    List<String> commands = new ArrayList<String>();
+    commands.add("/bin/bash");
+    commands.add(scriptFile.getAbsolutePath());
+    containerLaunchContext.setCommands(commands);
+    containerLaunchContext.setResource(recordFactory
+        .newRecordInstance(Resource.class));
+    containerLaunchContext.getResource().setMemory(1024);
+    StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
+    startRequest.setContainerLaunchContext(containerLaunchContext);
+    containerManager.startContainer(startRequest);
+    
+    GetContainerStatusRequest request =
+        recordFactory.newRecordInstance(GetContainerStatusRequest.class);
+        request.setContainerId(cId);
+    ContainerStatus containerStatus =
+        containerManager.getContainerStatus(request).getStatus();
+    Assert.assertEquals(ContainerState.RUNNING, containerStatus.getState());
+    
+    try {Thread.sleep(5000);} catch (InterruptedException ex) {ex.printStackTrace();} 
+    
+    nm.stop();
+    
+    // Now verify the contents of the file
+    // Script generates a message when it receives a sigterm
+    // so we look for that
+    BufferedReader reader =
+        new BufferedReader(new FileReader(processStartFile));
+
+    boolean foundSigTermMessage = false;
+    while (true) {
+      String line = reader.readLine();
+      if (line == null) {
+        break;
+      }
+      if (line.contains("SIGTERM")) {
+        foundSigTermMessage = true;
+        break;
+      }
+    }
+    Assert.assertTrue("Did not find sigterm message", foundSigTermMessage);
+    reader.close();
+  }
+  
+  private ContainerId createContainerId() {
+    ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
+    appId.setClusterTimestamp(0);
+    appId.setId(0);
+    ApplicationAttemptId appAttemptId = 
+        recordFactory.newRecordInstance(ApplicationAttemptId.class);
+    appAttemptId.setApplicationId(appId);
+    appAttemptId.setAttemptId(1);
+    ContainerId containerId = 
+        recordFactory.newRecordInstance(ContainerId.class);
+    containerId.setApplicationAttemptId(appAttemptId);
+    return containerId;
+  }
+  
+  private YarnConfiguration createNMConfig() {
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setInt(YarnConfiguration.NM_PMEM_MB, 5*1024); // 5GB
+    conf.set(YarnConfiguration.NM_ADDRESS, "127.0.0.1:12345");
+    conf.set(YarnConfiguration.NM_LOCALIZER_ADDRESS, "127.0.0.1:12346");
+    conf.set(YarnConfiguration.NM_LOG_DIRS, logsDir.getAbsolutePath());
+    conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogsDir.getAbsolutePath());
+    conf.set(YarnConfiguration.NM_LOCAL_DIRS, nmLocalDir.getAbsolutePath());
+    return conf;
+  }
+  
+  /**
+   * Creates a script to run a container that will run forever unless
+   * stopped by external means.
+   */
+  private File createUnhaltingScriptFile() throws IOException {
+    File scriptFile = new File(tmpDir, "scriptFile.sh");
+    BufferedWriter fileWriter = new BufferedWriter(new FileWriter(scriptFile));
+    fileWriter.write("#!/bin/bash\n\n");
+    fileWriter.write("echo \"Running testscript for delayed kill\"\n");
+    fileWriter.write("hello=\"Got SIGTERM\"\n");
+    fileWriter.write("umask 0\n");
+    fileWriter.write("trap \"echo $hello >> " + processStartFile + "\" SIGTERM\n");
+    fileWriter.write("echo \"Writing pid to start file\"\n");
+    fileWriter.write("echo $$ >> " + processStartFile + "\n");
+    fileWriter.write("while true; do\nsleep 1s;\ndone\n");
+
+    fileWriter.close();
+    return scriptFile;
+  }
+
+  private NodeManager getNodeManager() {
+    return new NodeManager() {
+      @Override
+      protected NodeStatusUpdater createNodeStatusUpdater(Context context,
+          Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
+        MockNodeStatusUpdater myNodeStatusUpdater = new MockNodeStatusUpdater(
+            context, dispatcher, healthChecker, metrics);
+        return myNodeStatusUpdater;
+      }
+    };
+  }
+}

+ 1 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java

@@ -36,7 +36,6 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.security.client.ClientTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
@@ -251,17 +250,12 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent> {
             YarnConfiguration.DEFAULT_APPLICATION_NAME);
       }
 
-      // Store application for recovery
-      ApplicationStore appStore = rmContext.getApplicationsStore()
-          .createApplicationStore(submissionContext.getApplicationId(),
-          submissionContext);
-
       // Create RMApp
       application =
           new RMAppImpl(applicationId, rmContext, this.conf,
             submissionContext.getApplicationName(),
             submissionContext.getUser(), submissionContext.getQueue(),
-            submissionContext, clientTokenStr, appStore, this.scheduler,
+            submissionContext, clientTokenStr, this.scheduler,
             this.masterService, submitTime);
 
       // Sanity check - duplicate?

+ 0 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java

@@ -23,8 +23,6 @@ import java.util.concurrent.ConcurrentMap;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.event.Dispatcher;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.NodeStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
@@ -41,10 +39,6 @@ public interface RMContext {
 
   Dispatcher getDispatcher();
 
-  NodeStore getNodeStore();
-
-  ApplicationsStore getApplicationsStore();
-
   ConcurrentMap<ApplicationId, RMApp> getRMApps();
   
   ConcurrentMap<String, RMNode> getInactiveRMNodes();

+ 1 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java

@@ -24,9 +24,6 @@ import java.util.concurrent.ConcurrentMap;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.event.Dispatcher;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.NodeStore;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
@@ -39,7 +36,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSe
 public class RMContextImpl implements RMContext {
 
   private final Dispatcher rmDispatcher;
-  private final Store store;
 
   private final ConcurrentMap<ApplicationId, RMApp> applications
     = new ConcurrentHashMap<ApplicationId, RMApp>();
@@ -58,7 +54,7 @@ public class RMContextImpl implements RMContext {
   private final RMContainerTokenSecretManager containerTokenSecretManager;
   private final ClientToAMTokenSecretManagerInRM clientToAMTokenSecretManager;
 
-  public RMContextImpl(Store store, Dispatcher rmDispatcher,
+  public RMContextImpl(Dispatcher rmDispatcher,
       ContainerAllocationExpirer containerAllocationExpirer,
       AMLivelinessMonitor amLivelinessMonitor,
       AMLivelinessMonitor amFinishingMonitor,
@@ -66,7 +62,6 @@ public class RMContextImpl implements RMContext {
       ApplicationTokenSecretManager appTokenSecretManager,
       RMContainerTokenSecretManager containerTokenSecretManager,
       ClientToAMTokenSecretManagerInRM clientTokenSecretManager) {
-    this.store = store;
     this.rmDispatcher = rmDispatcher;
     this.containerAllocationExpirer = containerAllocationExpirer;
     this.amLivelinessMonitor = amLivelinessMonitor;
@@ -82,16 +77,6 @@ public class RMContextImpl implements RMContext {
     return this.rmDispatcher;
   }
 
-  @Override
-  public NodeStore getNodeStore() {
-   return store;
-  }
-
-  @Override
-  public ApplicationsStore getApplicationsStore() {
-    return store;
-  }
-
   @Override
   public ConcurrentMap<ApplicationId, RMApp> getRMApps() {
     return this.applications;

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

@@ -46,8 +46,8 @@ import org.apache.hadoop.yarn.server.RMDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store.RMState;
+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.StoreFactory;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
@@ -119,12 +119,12 @@ public class ResourceManager extends CompositeService implements Recoverable {
   protected RMDelegationTokenSecretManager rmDTSecretManager;
   private WebApp webApp;
   protected RMContext rmContext;
-  private final Store store;
+  private final RMStateStore store;
   protected ResourceTrackerService resourceTracker;
 
   private Configuration conf;
 
-  public ResourceManager(Store store) {
+  public ResourceManager(RMStateStore store) {
     super("ResourceManager");
     this.store = store;
   }
@@ -161,7 +161,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
     this.containerTokenSecretManager = createContainerTokenSecretManager(conf);
     
     this.rmContext =
-        new RMContextImpl(this.store, this.rmDispatcher,
+        new RMContextImpl(this.rmDispatcher,
           this.containerAllocationExpirer, amLivelinessMonitor,
           amFinishingMonitor, tokenRenewer, this.appTokenSecretManager,
           this.containerTokenSecretManager, this.clientToAMSecretManager);
@@ -643,8 +643,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
 
   @Override
   public void recover(RMState state) throws Exception {
-    resourceTracker.recover(state);
-    scheduler.recover(state);
   }
   
   public static void main(String argv[]) {
@@ -652,14 +650,13 @@ public class ResourceManager extends CompositeService implements Recoverable {
     StringUtils.startupShutdownMessage(ResourceManager.class, argv, LOG);
     try {
       Configuration conf = new YarnConfiguration();
-      Store store =  StoreFactory.getStore(conf);
+      RMStateStore store =  StoreFactory.getStore(conf);
       ResourceManager resourceManager = new ResourceManager(store);
       ShutdownHookManager.get().addShutdownHook(
         new CompositeServiceShutdownHook(resourceManager),
         SHUTDOWN_HOOK_PRIORITY);
       resourceManager.init(conf);
       //resourceManager.recover(store.restore());
-      //store.doneWithRecovery();
       resourceManager.start();
     } catch (Throwable t) {
       LOG.fatal("Error starting ResourceManager", t);

+ 1 - 23
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java

@@ -44,7 +44,7 @@ import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store.RMState;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
@@ -297,28 +297,6 @@ public class ResourceTrackerService extends AbstractService implements
     return nodeHeartBeatResponse;
   }
 
-  public void recover(RMState state) {
-//
-//    List<RMNode> nodeManagers = state.getStoredNodeManagers();
-//    for (RMNode nm : nodeManagers) {
-//      createNewNode(nm.getNodeID(), nm.getNodeHostName(), nm
-//          .getCommandPort(), nm.getHttpPort(), nm.getNode(), nm
-//          .getTotalCapability());
-//    }
-//    for (Map.Entry<ApplicationId, ApplicationInfo> entry : state
-//        .getStoredApplications().entrySet()) {
-//      List<Container> containers = entry.getValue().getContainers();
-//      List<Container> containersToAdd = new ArrayList<Container>();
-//      for (Container c : containers) {
-//        RMNode containerNode = this.rmContext.getNodesCollection()
-//            .getNodeInfo(c.getNodeId());
-//        containersToAdd.add(c);
-//        containerNode.allocateContainer(entry.getKey(), containersToAdd);
-//        containersToAdd.clear();
-//      }
-//    }
-  }
-
   /**
    * resolving the network topology.
    * @param hostName the hostname of this node.

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

@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.server.resourcemanager.recovery;
-
-import java.io.IOException;
-
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationMaster;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.Container;
-
-public interface ApplicationsStore {
-  public ApplicationStore createApplicationStore(ApplicationId applicationId,
-      ApplicationSubmissionContext context) throws IOException;
-  public void removeApplication(ApplicationId application) throws IOException;
- 
-  public interface ApplicationStore {
-    public void storeContainer(Container container) throws IOException;
-    public void removeContainer(Container container) throws IOException;
-    public void storeMasterContainer(Container container) throws IOException;
-    public void updateApplicationState(ApplicationMaster master) throws IOException;
-    public boolean isLoggable();
-  }
-}

+ 2 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/NodeStore.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileRMStateStore.java

@@ -15,18 +15,8 @@
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */
-
 package org.apache.hadoop.yarn.server.resourcemanager.recovery;
 
-import java.io.IOException;
-
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
-
+public class FileRMStateStore implements RMStateStore {
 
-public interface NodeStore {
-  public void storeNode(RMNode node) throws IOException;
-  public void removeNode(RMNode node) throws IOException;
-  public NodeId getNextNodeId() throws IOException;
-  public boolean isLoggable();
-}
+}

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

@@ -1,128 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.server.resourcemanager.recovery;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationMaster;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
-
-public class MemStore implements Store {
-  RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
-  private NodeId nodeId;
-  private boolean doneWithRecovery = false;
-
-  public MemStore() {
-    nodeId = recordFactory.newRecordInstance(NodeId.class);
-    nodeId.setHost("TODO");
-    nodeId.setPort(-1);
-  }
-
-  @Override
-  public void storeNode(RMNode node) throws IOException {}
-
-  @Override
-  public void removeNode(RMNode node) throws IOException {}
-
-  private class ApplicationStoreImpl implements ApplicationStore {
-    @Override
-    public void storeContainer(Container container) throws IOException {}
-
-    @Override
-    public void removeContainer(Container container) throws IOException {}
-
-    @Override
-    public void storeMasterContainer(Container container) throws IOException {}
-
-    @Override
-    public void updateApplicationState(
-        ApplicationMaster master) throws IOException {}
-
-    @Override
-    public boolean isLoggable() {
-      return doneWithRecovery;
-    }
-
-  }
-
-  @Override
-  public ApplicationStore createApplicationStore(ApplicationId application,
-      ApplicationSubmissionContext context) throws IOException {
-    return new ApplicationStoreImpl();
-  }
-
-
-  @Override
-  public void removeApplication(ApplicationId application) throws IOException {}
-
-  @Override
-  public RMState restore() throws IOException {
-    MemRMState state = new MemRMState();
-    return state;
-  }
-
-  @Override
-  public synchronized NodeId getNextNodeId() throws IOException {
-    // TODO: FIXMEVinodkv
-//    int num = nodeId.getId();
-//    num++;
-//    nodeId.setId(num);
-    return nodeId;
-  }
-
-  private class MemRMState implements RMState {
-
-    public MemRMState() {
-      nodeId = recordFactory.newRecordInstance(NodeId.class);
-    }
-
-    @Override
-    public List<RMNode> getStoredNodeManagers()  {
-      return new ArrayList<RMNode>();
-    }
-
-    @Override
-    public NodeId getLastLoggedNodeId() {
-      return nodeId;
-    }
-
-    @Override
-    public Map<ApplicationId, ApplicationInfo> getStoredApplications() {
-      return new HashMap<ApplicationId, Store.ApplicationInfo>();
-    }
-  }
-
-  @Override
-  public boolean isLoggable() {
-    return doneWithRecovery;
-  }
-
-  @Override
-  public void doneWithRecovery() {
-    doneWithRecovery = true;
-  }
-}

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

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

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

@@ -17,7 +17,7 @@
 */
 package org.apache.hadoop.yarn.server.resourcemanager.recovery;
 
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store.RMState;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 
 public interface Recoverable {
   public void recover(RMState state) throws Exception;

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

@@ -1,46 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-package org.apache.hadoop.yarn.server.resourcemanager.recovery;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationMaster;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
-
-
-public interface Store extends NodeStore, ApplicationsStore {
-  public interface ApplicationInfo {
-    public ApplicationMaster getApplicationMaster();
-    public Container getMasterContainer();
-    public ApplicationSubmissionContext getApplicationSubmissionContext();
-    public List<Container> getContainers();
-  }
-  public interface RMState {
-    public List<RMNode> getStoredNodeManagers() ;
-    public Map<ApplicationId, ApplicationInfo> getStoredApplications();
-    public NodeId getLastLoggedNodeId();
-  }
-  public RMState restore() throws IOException;
-  public void doneWithRecovery();
-}

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

@@ -17,53 +17,17 @@
 */
 package org.apache.hadoop.yarn.server.resourcemanager.recovery;
 
-import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.yarn.api.records.ApplicationMaster;
-import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore;
 
 public class StoreFactory {
   
-  public static Store getStore(Configuration conf) {
-    Store store = ReflectionUtils.newInstance(
+  public static RMStateStore getStore(Configuration conf) {
+    RMStateStore store = ReflectionUtils.newInstance(
         conf.getClass(YarnConfiguration.RM_STORE, 
-            MemStore.class, Store.class), 
+            FileRMStateStore.class, RMStateStore.class), 
             conf);
     return store;
   }
-  
-  public static ApplicationStore createVoidAppStore() {
-    return new VoidApplicationStore();
-  }
-  
-  private static class VoidApplicationStore implements ApplicationStore {
-
-    public VoidApplicationStore() {}
-    
-    @Override
-    public void storeContainer(Container container) throws IOException {
-    }
-
-    @Override
-    public void removeContainer(Container container) throws IOException {
-    }
-
-    @Override
-    public void storeMasterContainer(Container container) throws IOException {
-    }
-
-    @Override
-    public void updateApplicationState(ApplicationMaster master)
-        throws IOException {
-    }
-
-    @Override
-    public boolean isLoggable() {
-      return false;
-    }
-  }
 }

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

@@ -1,509 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.server.resourcemanager.recovery;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationMaster;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.NodeReport;
-import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationMasterPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.NodeReportPBImpl;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationMasterProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
-import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.data.Stat;
-
-public class ZKStore implements Store {
-  private final Configuration conf;
-  private final ZooKeeper zkClient;
-  private static final Log LOG = LogFactory.getLog(ZKStore.class);
-  private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
-  private static final String NODES = "nodes/";
-  private static final String APPS = "apps/";
-  private static final String ZK_PATH_SEPARATOR = "/";
-  private static final String NODE_ID = "nodeid";
-  private static final String APP_MASTER = "master";
-  private static final String APP_MASTER_CONTAINER = "mastercontainer";
-  private final String ZK_ADDRESS;
-  private final int ZK_TIMEOUT;
-  private boolean doneWithRecovery = false;
-  
-  /** TODO make this generic **/
-  private NodeIdPBImpl nodeId = new NodeIdPBImpl();
-
-  /**
-   * TODO fix this for later to handle all kinds of events 
-   * of connection and session events.
-   *
-   */
-  private static class ZKWatcher implements Watcher {
-    @Override
-    public void process(WatchedEvent arg0) {
-    }
-  }
-
-  public ZKStore(Configuration conf) throws IOException {
-    this.conf = conf;
-    this.ZK_ADDRESS = conf.get(YarnConfiguration.RM_ZK_STORE_ADDRESS);
-    this.ZK_TIMEOUT = conf.getInt(YarnConfiguration.RM_ZK_STORE_TIMEOUT_MS,
-        YarnConfiguration.DEFAULT_RM_ZK_STORE_TIMEOUT_MS);
-    zkClient = new ZooKeeper(this.ZK_ADDRESS, 
-        this.ZK_TIMEOUT,
-        createZKWatcher() 
-    );
-    // TODO: FIXMEVinodkv
-//    this.nodeId.setId(0);
-  }
-
-  protected Watcher createZKWatcher() {
-    return new ZKWatcher();   
-  }
-
-  private NodeReportPBImpl createNodeManagerInfo(RMNode rmNode) {
-    NodeReport node = 
-      recordFactory.newRecordInstance(NodeReport.class);
-    node.setNodeId(rmNode.getNodeID());
-    node.setRackName(rmNode.getRackName());
-    node.setCapability(rmNode.getTotalCapability());
-    // TODO: FIXME
-//    node.setUsed(nodeInfo.getUsedResource());
-    // TODO: acm: refactor2 FIXME
-//  node.setNumContainers(rmNode.getNumContainers());
-    return (NodeReportPBImpl)node;
-  }
-
-  @Override
-  public synchronized void storeNode(RMNode node) throws IOException {
-    /** create a storage node and store it in zk **/
-    if (!doneWithRecovery) return;
-    // TODO: FIXMEVinodkv
-//    NodeReportPBImpl nodeManagerInfo = createNodeManagerInfo(node);
-//    byte[] bytes = nodeManagerInfo.getProto().toByteArray();
-//    try {
-//      zkClient.create(NODES + Integer.toString(node.getNodeID().getId()), bytes, null,
-//          CreateMode.PERSISTENT);
-//    } catch(InterruptedException ie) {
-//      LOG.info("Interrupted", ie);
-//      throw new InterruptedIOException("Interrupted");
-//    } catch(KeeperException ke) {
-//      LOG.info("Keeper exception", ke);
-//      throw convertToIOException(ke);
-//    }
-  }
-
-  @Override
-  public synchronized void removeNode(RMNode node) throws IOException {
-    if (!doneWithRecovery) return;
-    
-//    TODO: FIXME VINODKV
-//    /** remove a storage node **/
-//    try {
-//      zkClient.delete(NODES + Integer.toString(node.getNodeID().getId()), -1);
-//    } catch(InterruptedException ie) {
-//      LOG.info("Interrupted", ie);
-//      throw new InterruptedIOException("Interrupted");
-//    } catch(KeeperException ke) {
-//      LOG.info("Keeper exception", ke);
-//      throw convertToIOException(ke);
-//    }
-
-  }
-
-  private static IOException convertToIOException(KeeperException ke) {
-    IOException io = new IOException();
-    io.setStackTrace(ke.getStackTrace());
-    return io;
-  }
-
-  @Override
-  public synchronized NodeId getNextNodeId() throws IOException {
-//    TODO: FIXME VINODKV
-//    int num = nodeId.getId();
-//    num++;
-//    nodeId.setId(num);
-//    try {
-//      zkClient.setData(NODES + NODE_ID, nodeId.getProto().toByteArray() , -1);
-//    } catch(InterruptedException ie) {
-//      LOG.info("Interrupted", ie);
-//      throw new InterruptedIOException(ie.getMessage());
-//    } catch(KeeperException ke) {
-//      throw convertToIOException(ke);
-//    }
-    return nodeId;
-  }
-
-  private String containerPathFromContainerId(ContainerId containerId) {
-    String appString = ConverterUtils.toString(
-        containerId.getApplicationAttemptId().getApplicationId());
-    return appString + "/" + containerId.getId();
-  }
-
-  private class ZKApplicationStore implements ApplicationStore {
-    private final ApplicationId applicationId;
-
-    public ZKApplicationStore(ApplicationId applicationId) {
-      this.applicationId = applicationId;
-    }
-
-    @Override
-    public void storeMasterContainer(Container container) throws IOException {
-      if (!doneWithRecovery) return;
-      
-      ContainerPBImpl containerPBImpl = (ContainerPBImpl) container;
-      try {
-        zkClient.setData(APPS + 
-            ConverterUtils.toString(
-                container.getId().getApplicationAttemptId().getApplicationId()) 
-                +
-            ZK_PATH_SEPARATOR + APP_MASTER_CONTAINER
-            , containerPBImpl.getProto().toByteArray(), -1);
-      } catch(InterruptedException ie) {
-        LOG.info("Interrupted", ie);
-        throw new InterruptedIOException(ie.getMessage());
-      } catch(KeeperException ke) {
-        LOG.info("Keeper exception", ke);
-        throw convertToIOException(ke);
-      }
-    }
-    @Override
-    public synchronized void storeContainer(Container container) throws IOException {
-      if (!doneWithRecovery) return;
-      
-      ContainerPBImpl containerPBImpl = (ContainerPBImpl) container;
-      try {
-        zkClient.create(APPS + containerPathFromContainerId(container.getId())
-            , containerPBImpl.getProto().toByteArray(), null, CreateMode.PERSISTENT);
-      } catch(InterruptedException ie) {
-        LOG.info("Interrupted", ie);
-        throw new InterruptedIOException(ie.getMessage());
-      } catch(KeeperException ke) {
-        LOG.info("Keeper exception", ke);
-        throw convertToIOException(ke);
-      }
-    }
-
-    @Override
-    public synchronized void removeContainer(Container container) throws IOException {
-      if (!doneWithRecovery) return;
-      try { 
-        zkClient.delete(APPS + containerPathFromContainerId(container.getId()),
-            -1);
-      } catch(InterruptedException ie) {
-        throw new InterruptedIOException(ie.getMessage());
-      } catch(KeeperException ke) {
-        LOG.info("Keeper exception", ke);
-        throw convertToIOException(ke);
-      }
-    }
-
-    @Override
-    public void updateApplicationState(
-        ApplicationMaster master) throws IOException {
-      if (!doneWithRecovery) return;
-      
-      String appString = APPS + ConverterUtils.toString(applicationId);
-      ApplicationMasterPBImpl masterPBImpl = (ApplicationMasterPBImpl) master;
-      try {
-        zkClient.setData(appString, masterPBImpl.getProto().toByteArray(), -1);
-      } catch(InterruptedException ie) {
-        LOG.info("Interrupted", ie);
-        throw new InterruptedIOException(ie.getMessage());
-      } catch(KeeperException ke) {
-        LOG.info("Keeper exception", ke);
-        throw convertToIOException(ke);
-      }
-    }
-
-    @Override
-    public boolean isLoggable() {
-      return doneWithRecovery;
-    }
-  }
-
-  @Override
-  public synchronized ApplicationStore createApplicationStore(ApplicationId application, 
-      ApplicationSubmissionContext context) throws IOException {
-    if (!doneWithRecovery) return new ZKApplicationStore(application);
-    
-    ApplicationSubmissionContextPBImpl contextPBImpl = (ApplicationSubmissionContextPBImpl) context;
-    String appString = APPS + ConverterUtils.toString(application);
-   
-    ApplicationMasterPBImpl masterPBImpl = new ApplicationMasterPBImpl();
-    ContainerPBImpl container = new ContainerPBImpl();
-    try {
-      zkClient.create(appString, contextPBImpl.getProto()
-          .toByteArray(), null, CreateMode.PERSISTENT);
-      zkClient.create(appString + ZK_PATH_SEPARATOR + APP_MASTER, 
-          masterPBImpl.getProto().toByteArray(), null, CreateMode.PERSISTENT);
-      zkClient.create(appString + ZK_PATH_SEPARATOR + APP_MASTER_CONTAINER, 
-          container.getProto().toByteArray(), null, CreateMode.PERSISTENT);
-    } catch(InterruptedException ie) {
-      LOG.info("Interrupted", ie);
-      throw new InterruptedIOException(ie.getMessage());
-    } catch(KeeperException ke) {
-      LOG.info("Keeper exception", ke);
-      throw convertToIOException(ke);
-    }
-    return new ZKApplicationStore(application);
-  }
-
-  @Override
-  public synchronized void removeApplication(ApplicationId application) throws IOException {
-    if (!doneWithRecovery) return;
-    
-    try {
-      zkClient.delete(APPS + ConverterUtils.toString(application), -1);
-    } catch(InterruptedException ie) {
-      LOG.info("Interrupted", ie);
-      throw new InterruptedIOException(ie.getMessage());
-    } catch(KeeperException ke) {
-      LOG.info("Keeper Exception", ke);
-      throw convertToIOException(ke);
-    }
-  }
-
-  @Override
-  public boolean isLoggable() {
-    return doneWithRecovery;
-  }
-
-  @Override
-  public void doneWithRecovery() {
-    this.doneWithRecovery = true;
-  }
-
-  
-  @Override
-  public synchronized RMState restore() throws IOException {
-    ZKRMState rmState = new ZKRMState();
-    rmState.load();
-    return rmState;
-  }  
-
-  private static class ApplicationInfoImpl implements ApplicationInfo {
-    private ApplicationMaster master;
-    private Container masterContainer;
-
-    private final ApplicationSubmissionContext context;
-    private final List<Container> containers = new ArrayList<Container>();
-
-    public ApplicationInfoImpl(ApplicationSubmissionContext context) {
-      this.context = context;
-    }
-
-    public void setApplicationMaster(ApplicationMaster master) {
-      this.master = master;
-    }
-
-    public void setMasterContainer(Container container) {
-      this.masterContainer = container;
-    }
-
-    @Override
-    public ApplicationMaster getApplicationMaster() {
-      return this.master;
-    }
-
-    @Override
-    public ApplicationSubmissionContext getApplicationSubmissionContext() {
-      return this.context;
-    }
-
-    @Override
-    public Container getMasterContainer() {
-      return this.masterContainer;
-    }
-
-    @Override
-    public List<Container> getContainers() {
-      return this.containers;
-    }
-
-    public void addContainer(Container container) {
-      containers.add(container);
-    }
-  }
-
-  private class ZKRMState implements RMState {
-    private List<RMNode> nodeManagers = new ArrayList<RMNode>();
-    private Map<ApplicationId, ApplicationInfo> applications = new 
-    HashMap<ApplicationId, ApplicationInfo>();
-
-    public ZKRMState() {
-      LOG.info("Restoring RM state from ZK");
-    }
-
-    private synchronized List<NodeReport> listStoredNodes() throws IOException {
-      /** get the list of nodes stored in zk **/
-      //TODO PB
-      List<NodeReport> nodes = new ArrayList<NodeReport>();
-      Stat stat = new Stat();
-      try {
-        List<String> children = zkClient.getChildren(NODES, false);
-        for (String child: children) {
-          byte[] data = zkClient.getData(NODES + child, false, stat);
-          NodeReportPBImpl nmImpl = new NodeReportPBImpl(
-              NodeReportProto.parseFrom(data));
-          nodes.add(nmImpl);
-        }
-      } catch (InterruptedException ie) {
-        LOG.info("Interrupted" , ie);
-        throw new InterruptedIOException("Interrupted");
-      } catch(KeeperException ke) {
-        LOG.error("Failed to list nodes", ke);
-        throw convertToIOException(ke);
-      }
-      return nodes;
-    }
-
-    @Override
-    public List<RMNode> getStoredNodeManagers()  {
-      return nodeManagers;
-    }
-
-    @Override
-    public NodeId getLastLoggedNodeId() {
-      return nodeId;
-    }
-
-    private void readLastNodeId() throws IOException {
-      Stat stat = new Stat();
-      try {
-        byte[] data = zkClient.getData(NODES + NODE_ID, false, stat);
-        nodeId = new NodeIdPBImpl(NodeIdProto.parseFrom(data));
-      } catch(InterruptedException ie) {
-        LOG.info("Interrupted", ie);
-        throw new InterruptedIOException(ie.getMessage());
-      } catch(KeeperException ke) {
-        LOG.info("Keeper Exception", ke);
-        throw convertToIOException(ke);
-      }
-    }
-
-    private ApplicationInfo getAppInfo(String app) throws IOException {
-      ApplicationInfoImpl info = null;
-      Stat stat = new Stat();
-      try {
-        ApplicationSubmissionContext context = null;
-        byte[] data = zkClient.getData(APPS + app, false, stat);
-        context = new ApplicationSubmissionContextPBImpl(
-            ApplicationSubmissionContextProto.parseFrom(data));
-        info = new ApplicationInfoImpl(context);
-        List<String> children = zkClient.getChildren(APPS + app, false, stat);
-        ApplicationMaster master = null;
-        for (String child: children) {
-          byte[] childdata = zkClient.getData(APPS + app + ZK_PATH_SEPARATOR + child, false, stat);
-          if (APP_MASTER.equals(child)) {
-            master = new ApplicationMasterPBImpl(ApplicationMasterProto.parseFrom(childdata));
-            info.setApplicationMaster(master);
-          } else if (APP_MASTER_CONTAINER.equals(child)) {
-            Container masterContainer = new ContainerPBImpl(ContainerProto.parseFrom(data));
-            info.setMasterContainer(masterContainer);
-          } else {
-            Container container = new ContainerPBImpl(ContainerProto.parseFrom(data));
-            info.addContainer(container);
-          }
-        }
-      } catch(InterruptedException ie) {
-        LOG.info("Interrupted", ie);
-        throw new InterruptedIOException(ie.getMessage());
-      } catch(KeeperException ke) {
-        throw convertToIOException(ke);
-      }
-      return info;
-    }
-
-    private void load() throws IOException {
-      List<NodeReport> nodeInfos = listStoredNodes();
-      final Pattern trackerPattern = Pattern.compile(".*:.*");
-      final Matcher m = trackerPattern.matcher("");
-      for (NodeReport node: nodeInfos) {
-        m.reset(node.getNodeId().getHost());
-        if (!m.find()) {
-          LOG.info("Skipping node, bad node-address "
-              + node.getNodeId().getHost());
-          continue;
-        }
-        String hostName = m.group(0);
-        int cmPort = Integer.valueOf(m.group(1));
-        m.reset(node.getHttpAddress());
-        if (!m.find()) {
-          LOG.info("Skipping node, bad http-address " + node.getHttpAddress());
-          continue;
-        }
-        int httpPort = Integer.valueOf(m.group(1));
-        // TODO: FindBugs warns passing null below. Commenting this for later.
-//        RMNode nm = new RMNodeImpl(node.getNodeId(), null,
-//            hostName, cmPort, httpPort,
-//            ResourceTrackerService.resolve(node.getNodeId().getHost()), 
-//            node.getCapability());
-//        nodeManagers.add(nm);
-      }
-      readLastNodeId();
-      /* make sure we get all the applications */
-      List<String> apps = null;
-      try {
-        apps = zkClient.getChildren(APPS, false);
-      } catch(InterruptedException ie) {
-        LOG.info("Interrupted", ie);
-        throw new InterruptedIOException(ie.getMessage());
-      } catch(KeeperException ke) {
-        throw convertToIOException(ke);
-      }
-      for (String app: apps) {
-        ApplicationInfo info = getAppInfo(app);
-        applications.put(info.getApplicationMaster().getApplicationId(), info);
-      }
-    }
-
-    @Override
-    public Map<ApplicationId, ApplicationInfo> getStoredApplications() {
-      return applications;
-    }
-  }
-}

+ 23 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/package-info.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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.resourcemanager.recovery;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

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

@@ -29,7 +29,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 
@@ -131,13 +130,6 @@ public interface RMApp extends EventHandler<RMAppEvent> {
    */
   int pullRMNodeUpdates(Collection<RMNode> updatedNodes);
 
-  /**
-   * Application level metadata is stored in {@link ApplicationStore} which
-   * can persist the information.
-   * @return the {@link ApplicationStore}  for this {@link RMApp}.
-   */
-  ApplicationStore getApplicationStore();
-
   /**
    * The finish time of the {@link RMApp}
    * @return the finish time of the application.,

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

@@ -49,7 +49,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppNodeUpdateEvent.RMAppNodeUpdateType;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -81,7 +80,6 @@ public class RMAppImpl implements RMApp {
   private final String name;
   private final ApplicationSubmissionContext submissionContext;
   private final String clientTokenStr;
-  private final ApplicationStore appStore;
   private final Dispatcher dispatcher;
   private final YarnScheduler scheduler;
   private final ApplicationMasterService masterService;
@@ -213,7 +211,6 @@ public class RMAppImpl implements RMApp {
   public RMAppImpl(ApplicationId applicationId, RMContext rmContext,
       Configuration config, String name, String user, String queue,
       ApplicationSubmissionContext submissionContext, String clientTokenStr,
-      ApplicationStore appStore,
       YarnScheduler scheduler, ApplicationMasterService masterService, 
       long submitTime) {
 
@@ -227,7 +224,6 @@ public class RMAppImpl implements RMApp {
     this.queue = queue;
     this.submissionContext = submissionContext;
     this.clientTokenStr = clientTokenStr;
-    this.appStore = appStore;
     this.scheduler = scheduler;
     this.masterService = masterService;
     this.submitTime = submitTime;
@@ -340,11 +336,6 @@ public class RMAppImpl implements RMApp {
     }
   }
 
-  @Override
-  public ApplicationStore getApplicationStore() {
-    return this.appStore;
-  }
-
   private YarnApplicationState createApplicationState(RMAppState rmAppState) {
     switch(rmAppState) {
     case NEW:

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

@@ -33,6 +33,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -399,7 +400,7 @@ public class RMAppAttemptImpl implements RMAppAttempt {
       final String trackingUriWithoutScheme) {
     this.readLock.lock();
     try {
-      URI trackingUri = trackingUriWithoutScheme == null ? null :
+      URI trackingUri = StringUtils.isEmpty(trackingUriWithoutScheme) ? null :
         ProxyUriUtils.getUriFromAMUrl(trackingUriWithoutScheme);
       String proxy = YarnConfiguration.getProxyHostAndPort(conf);
       URI proxyUri = ProxyUriUtils.getUriFromAMUrl(proxy);

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

@@ -36,7 +36,6 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
@@ -70,14 +69,12 @@ public class AppSchedulingInfo {
   boolean pending = true; // for app metrics
 
   public AppSchedulingInfo(ApplicationAttemptId appAttemptId,
-      String user, Queue queue, ActiveUsersManager activeUsersManager,
-      ApplicationStore store) {
+      String user, Queue queue, ActiveUsersManager activeUsersManager) {
     this.applicationAttemptId = appAttemptId;
     this.applicationId = appAttemptId.getApplicationId();
     this.queue = queue;
     this.queueName = queue.getQueueName();
     this.user = user;
-    //this.store = store;
     this.activeUsersManager = activeUsersManager;
   }
 

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

@@ -49,7 +49,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store.RMState;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
@@ -365,7 +365,7 @@ implements ResourceScheduler, CapacitySchedulerContext, Configurable {
     // TODO: Fix store
     FiCaSchedulerApp SchedulerApp = 
         new FiCaSchedulerApp(applicationAttemptId, user, queue, 
-            queue.getActiveUsersManager(), rmContext, null);
+            queue.getActiveUsersManager(), rmContext);
 
     // Submit to the queue
     try {
@@ -767,18 +767,7 @@ implements ResourceScheduler, CapacitySchedulerContext, Configurable {
   @Override
   @Lock(Lock.NoLock.class)
   public void recover(RMState state) throws Exception {
-    // TODO: VINDOKVFIXME recovery
-//    applications.clear();
-//    for (Map.Entry<ApplicationId, ApplicationInfo> entry : state.getStoredApplications().entrySet()) {
-//      ApplicationId appId = entry.getKey();
-//      ApplicationInfo appInfo = entry.getValue();
-//      SchedulerApp app = applications.get(appId);
-//      app.allocate(appInfo.getContainers());
-//      for (Container c: entry.getValue().getContainers()) {
-//        Queue queue = queues.get(appInfo.getApplicationSubmissionContext().getQueue());
-//        queue.recoverContainer(clusterResource, applications.get(appId), c);
-//      }
-//    }
+    // NOT IMPLEMENTED
   }
 
   @Override

+ 2 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java

@@ -43,7 +43,6 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -111,11 +110,11 @@ public class FiCaSchedulerApp extends SchedulerApplication {
   private final RMContext rmContext;
   public FiCaSchedulerApp(ApplicationAttemptId applicationAttemptId, 
       String user, Queue queue, ActiveUsersManager activeUsersManager,
-      RMContext rmContext, ApplicationStore store) {
+      RMContext rmContext) {
     this.rmContext = rmContext;
     this.appSchedulingInfo = 
         new AppSchedulingInfo(applicationAttemptId, user, queue,  
-            activeUsersManager, store);
+            activeUsersManager);
     this.queue = queue;
   }
 

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

@@ -50,10 +50,10 @@ public class AppSchedulable extends Schedulable {
   private long startTime;
   private static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
   private static final Log LOG = LogFactory.getLog(AppSchedulable.class);
-  private FSQueue queue;
+  private FSLeafQueue queue;
   private RMContainerTokenSecretManager containerTokenSecretManager;
 
-  public AppSchedulable(FairScheduler scheduler, FSSchedulerApp app, FSQueue queue) {
+  public AppSchedulable(FairScheduler scheduler, FSSchedulerApp app, FSLeafQueue queue) {
     this.scheduler = scheduler;
     this.app = app;
     this.startTime = System.currentTimeMillis();
@@ -96,9 +96,6 @@ public class AppSchedulable extends Schedulable {
     return startTime;
   }
 
-  @Override
-  public void redistributeShare() {}
-
   @Override
   public Resource getResourceUsage() {
     return app.getCurrentConsumption();
@@ -114,7 +111,7 @@ public class AppSchedulable extends Schedulable {
    * Get metrics reference from containing queue.
    */
   public QueueMetrics getMetrics() {
-    return queue.getQueueSchedulable().getMetrics();
+    return queue.getMetrics();
   }
 
   @Override

+ 88 - 166
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueueSchedulable.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java

@@ -22,73 +22,57 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
-import java.util.HashMap;
 import java.util.Iterator;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
-import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
-import org.apache.hadoop.yarn.api.records.QueueInfo;
-import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 
-@Private
-@Unstable
-public class FSQueueSchedulable extends Schedulable implements Queue {
-  public static final Log LOG = LogFactory.getLog(
-      FSQueueSchedulable.class.getName());
-
-  private FairScheduler scheduler;
-  private FSQueue queue;
-  private QueueManager queueMgr;
-  private List<AppSchedulable> appScheds = new LinkedList<AppSchedulable>();
+public class FSLeafQueue extends FSQueue {
+  private static final Log LOG = LogFactory.getLog(
+      FSLeafQueue.class.getName());
+    
+  private final List<AppSchedulable> appScheds = 
+      new ArrayList<AppSchedulable>();
+
+  /** Scheduling mode for jobs inside the queue (fair or FIFO) */
+  private SchedulingMode schedulingMode;
+  
+  private final FairScheduler scheduler;
+  private final QueueManager queueMgr;
   private Resource demand = Resources.createResource(0);
-  private QueueMetrics metrics;
-  private RecordFactory recordFactory =
-      RecordFactoryProvider.getRecordFactory(null);
-
+  
   // Variables used for preemption
-  long lastTimeAtMinShare;
-  long lastTimeAtHalfFairShare;
-
-  // Constructor for tests
-  protected FSQueueSchedulable(FairScheduler scheduler, FSQueue fsQueue,
-      QueueManager qMgr, QueueMetrics metrics, long minShare, long fairShare) {
-    this.scheduler = scheduler;
-    this.queueMgr = qMgr;
-    this.queue = fsQueue;
-    this.metrics = metrics;
-    this.lastTimeAtMinShare = minShare;
-    this.lastTimeAtHalfFairShare = fairShare;
-  }
-
-  public FSQueueSchedulable(FairScheduler scheduler, FSQueue queue) {
+  private long lastTimeAtMinShare;
+  private long lastTimeAtHalfFairShare;
+  
+  public FSLeafQueue(String name, QueueManager queueMgr, FairScheduler scheduler,
+      FSParentQueue parent) {
+    super(name, queueMgr, scheduler, parent);
     this.scheduler = scheduler;
-    this.queue = queue;
-    this.queueMgr = scheduler.getQueueManager();
-    this.metrics = QueueMetrics.forQueue(getName(), null, true, scheduler.getConf());
+    this.queueMgr = queueMgr;
     this.lastTimeAtMinShare = scheduler.getClock().getTime();
     this.lastTimeAtHalfFairShare = scheduler.getClock().getTime();
   }
-
-  public void addApp(AppSchedulable app) {
-    appScheds.add(app);
+  
+  public void addApp(FSSchedulerApp app) {
+    AppSchedulable appSchedulable = new AppSchedulable(scheduler, app, this);
+    app.setAppSchedulable(appSchedulable);
+    appScheds.add(appSchedulable);
   }
-
+  
+  // for testing
+  void addAppSchedulable(AppSchedulable appSched) {
+    appScheds.add(appSched);
+  }
+  
   public void removeApp(FSSchedulerApp app) {
     for (Iterator<AppSchedulable> it = appScheds.iterator(); it.hasNext();) {
       AppSchedulable appSched = it.next();
@@ -98,17 +82,47 @@ public class FSQueueSchedulable extends Schedulable implements Queue {
       }
     }
   }
+  
+  public Collection<AppSchedulable> getAppSchedulables() {
+    return appScheds;
+  }
+
+  public void setSchedulingMode(SchedulingMode mode) {
+    this.schedulingMode = mode;
+  }
+  
+  @Override
+  public void recomputeFairShares() {
+    if (schedulingMode == SchedulingMode.FAIR) {
+      SchedulingAlgorithms.computeFairShares(appScheds, getFairShare());
+    } else {
+      for (AppSchedulable sched: appScheds) {
+        sched.setFairShare(Resources.createResource(0));
+      }
+    }
+  }
+
+  @Override
+  public Resource getDemand() {
+    return demand;
+  }
+
+  @Override
+  public Resource getResourceUsage() {
+    Resource usage = Resources.createResource(0);
+    for (AppSchedulable app : appScheds) {
+      Resources.addTo(usage, app.getResourceUsage());
+    }
+    return usage;
+  }
 
-  /**
-   * Update demand by asking apps in the queue to update
-   */
   @Override
   public void updateDemand() {
     // Compute demand by iterating through apps in the queue
     // Limit demand to maxResources
-    Resource maxRes = queueMgr.getMaxResources(queue.getName());
+    Resource maxRes = queueMgr.getMaxResources(getName());
     demand = Resources.createResource(0);
-    for (AppSchedulable sched: appScheds) {
+    for (AppSchedulable sched : appScheds) {
       sched.updateDemand();
       Resource toAdd = sched.getDemand();
       if (LOG.isDebugEnabled()) {
@@ -128,46 +142,12 @@ public class FSQueueSchedulable extends Schedulable implements Queue {
     }
   }
 
-  /**
-   * Distribute the queue's fair share among its jobs
-   */
-  @Override
-  public void redistributeShare() {
-    if (queue.getSchedulingMode() == SchedulingMode.FAIR) {
-      SchedulingAlgorithms.computeFairShares(appScheds, getFairShare());
-    } else {
-      for (AppSchedulable sched: appScheds) {
-        sched.setFairShare(Resources.createResource(0));
-      }
-    }
-  }
-
-  @Override
-  public Resource getDemand() {
-    return demand;
-  }
-
-  @Override
-  public Resource getMinShare() {
-    return queueMgr.getMinResources(queue.getName());
-  }
-
-  @Override
-  public double getWeight() {
-    return queueMgr.getQueueWeight(queue.getName());
-  }
-
-  @Override
-  public long getStartTime() {
-    return 0;
-  }
-
   @Override
   public Resource assignContainer(FSSchedulerNode node, boolean reserved) {
     LOG.debug("Node offered to queue: " + getName() + " reserved: " + reserved);
     // If this queue is over its limit, reject
     if (Resources.greaterThan(getResourceUsage(),
-        queueMgr.getMaxResources(queue.getName()))) {
+        queueMgr.getMaxResources(getName()))) {
       return Resources.none();
     }
 
@@ -185,15 +165,14 @@ public class FSQueueSchedulable extends Schedulable implements Queue {
 
     // Otherwise, chose app to schedule based on given policy (fair vs fifo).
     else {
-      SchedulingMode mode = queue.getSchedulingMode();
-
       Comparator<Schedulable> comparator;
-      if (mode == SchedulingMode.FIFO) {
+      if (schedulingMode == SchedulingMode.FIFO) {
         comparator = new SchedulingAlgorithms.FifoComparator();
-      } else if (mode == SchedulingMode.FAIR) {
+      } else if (schedulingMode == SchedulingMode.FAIR) {
         comparator = new SchedulingAlgorithms.FairShareComparator();
       } else {
-        throw new RuntimeException("Unsupported queue scheduling mode " + mode);
+        throw new RuntimeException("Unsupported queue scheduling mode " + 
+            schedulingMode);
       }
 
       Collections.sort(appScheds, comparator);
@@ -203,81 +182,13 @@ public class FSQueueSchedulable extends Schedulable implements Queue {
 
       return Resources.none();
     }
-
-  }
-
-  @Override
-  public String getName() {
-    return queue.getName();
-  }
-
-  FSQueue getQueue() {
-    return queue;
-  }
-
-  public Collection<AppSchedulable> getAppSchedulables() {
-    return appScheds;
-  }
-
-  public long getLastTimeAtMinShare() {
-    return lastTimeAtMinShare;
-  }
-
-  public void setLastTimeAtMinShare(long lastTimeAtMinShare) {
-    this.lastTimeAtMinShare = lastTimeAtMinShare;
-  }
-
-  public long getLastTimeAtHalfFairShare() {
-    return lastTimeAtHalfFairShare;
-  }
-
-  public void setLastTimeAtHalfFairShare(long lastTimeAtHalfFairShare) {
-    this.lastTimeAtHalfFairShare = lastTimeAtHalfFairShare;
-  }
-
-  @Override
-  public QueueMetrics getMetrics() {
-    return metrics;
-  }
-
-  @Override
-  public Resource getResourceUsage() {
-    Resource usage = Resources.createResource(0);
-    for (AppSchedulable app : appScheds) {
-      Resources.addTo(usage, app.getResourceUsage());
-    }
-    return usage;
   }
 
   @Override
-  public Priority getPriority() {
-    Priority p = recordFactory.newRecordInstance(Priority.class);
-    p.setPriority(1);
-    return p;
-  }
-
-  @Override
-  public Map<QueueACL, AccessControlList> getQueueAcls() {
-    Map<QueueACL, AccessControlList> acls = queueMgr.getQueueAcls(getName());
-    return new HashMap<QueueACL, AccessControlList>(acls);
-  }
-
-  @Override
-  public QueueInfo getQueueInfo(boolean includeChildQueues, boolean recursive) {
-    QueueInfo queueInfo = recordFactory.newRecordInstance(QueueInfo.class);
-    queueInfo.setQueueName(getQueueName());
-    // TODO: we might change these queue metrics around a little bit
-    // to match the semantics of the fair scheduler.
-    queueInfo.setCapacity((float) getFairShare().getMemory() /
-        scheduler.getClusterCapacity().getMemory());
-    queueInfo.setCapacity((float) getResourceUsage().getMemory() /
-        scheduler.getClusterCapacity().getMemory());
-
-    queueInfo.setChildQueues(new ArrayList<QueueInfo>());
-    queueInfo.setQueueState(QueueState.RUNNING);
-    return queueInfo;
+  public Collection<FSQueue> getChildQueues() {
+    return new ArrayList<FSQueue>(1);
   }
-
+  
   @Override
   public List<QueueUserACLInfo> getQueueUserAclInfo(UserGroupInformation user) {
     QueueUserACLInfo userAclInfo =
@@ -294,9 +205,20 @@ public class FSQueueSchedulable extends Schedulable implements Queue {
     userAclInfo.setUserAcls(operations);
     return Collections.singletonList(userAclInfo);
   }
+  
+  public long getLastTimeAtMinShare() {
+    return lastTimeAtMinShare;
+  }
 
-  @Override
-  public String getQueueName() {
-    return getName();
+  public void setLastTimeAtMinShare(long lastTimeAtMinShare) {
+    this.lastTimeAtMinShare = lastTimeAtMinShare;
+  }
+
+  public long getLastTimeAtHalfFairShare() {
+    return lastTimeAtHalfFairShare;
+  }
+
+  public void setLastTimeAtHalfFairShare(long lastTimeAtHalfFairShare) {
+    this.lastTimeAtHalfFairShare = lastTimeAtHalfFairShare;
   }
 }

Some files were not shown because too many files changed in this diff