浏览代码

Merge r1555021 through r1561510 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5535@1561516 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 11 年之前
父节点
当前提交
a9110e1788
共有 100 个文件被更改,包括 4128 次插入993 次删除
  1. 7 0
      hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
  2. 15 3
      hadoop-common-project/hadoop-common/CHANGES.txt
  3. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  4. 10 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
  5. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java
  6. 6 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableFactories.java
  7. 3 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
  8. 5 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFS.java
  9. 11 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
  10. 3 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
  11. 12 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
  12. 1 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
  13. 3 3
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSKerberosAuthenticator.java
  14. 7 3
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java
  15. 34 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpsFSFileSystem.java
  16. 1 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebApp.java
  17. 8 5
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/DelegationTokenIdentifier.java
  18. 14 5
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/security/DelegationTokenManagerService.java
  19. 9 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/ServerWebApp.java
  20. 21 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh
  21. 3 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/sbin/httpfs.sh
  22. 135 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml
  23. 42 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/ServerSetup.apt.vm
  24. 6 2
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
  25. 99 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithSWebhdfsFileSystem.java
  26. 27 5
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSKerberosAuthenticationHandler.java
  27. 9 3
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/security/TestDelegationTokenManagerService.java
  28. 43 14
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java
  29. 238 204
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  30. 5 0
      hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
  31. 33 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
  32. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperAsHASharedDir.java
  33. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  34. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  35. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  36. 44 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  37. 58 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
  38. 14 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java
  39. 68 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
  40. 68 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
  41. 138 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
  42. 14 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java
  43. 94 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
  44. 92 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
  45. 18 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/GetJournalEditServlet.java
  46. 9 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java
  47. 65 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
  48. 28 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
  49. 32 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
  50. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  51. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  52. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java
  53. 77 132
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
  54. 119 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
  55. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
  56. 19 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
  57. 33 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
  58. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
  59. 23 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  60. 58 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  61. 88 114
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  62. 52 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  63. 47 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
  64. 51 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
  65. 54 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
  66. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
  67. 174 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNUpgradeUtil.java
  68. 28 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  69. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
  70. 20 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
  71. 78 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
  72. 200 217
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
  73. 113 55
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
  74. 20 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css
  75. 46 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSHighAvailabilityWithQJM.apt.vm
  76. 51 17
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  77. 20 25
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
  78. 34 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
  79. 59 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
  80. 9 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniJournalCluster.java
  81. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
  82. 44 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java
  83. 25 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
  84. 71 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
  85. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java
  86. 674 49
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java
  87. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
  88. 64 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotFileLength.java
  89. 11 0
      hadoop-mapreduce-project/CHANGES.txt
  90. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
  91. 19 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  92. 13 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java
  93. 9 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java
  94. 87 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java
  95. 3 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLogAppender.java
  96. 29 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
  97. 14 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestNetworkedJob.java
  98. 0 1
      hadoop-mapreduce-project/pom.xml
  99. 0 4
      hadoop-project/pom.xml
  100. 0 1
      hadoop-tools/hadoop-distcp/pom.xml

+ 7 - 0
hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml

@@ -122,6 +122,13 @@
         <include>*-sources.jar</include>
       </includes>
     </fileSet>
+    <fileSet>
+      <directory>hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/target</directory>
+      <outputDirectory>/share/hadoop/${hadoop.component}/sources</outputDirectory>
+      <includes>
+        <include>*-sources.jar</include>
+      </includes>
+    </fileSet>
     <fileSet>
       <directory>hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target</directory>
       <outputDirectory>/share/hadoop/${hadoop.component}/sources</outputDirectory>

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

@@ -421,6 +421,12 @@ Release 2.4.0 - UNRELEASED
     HADOOP-9420. Add percentile or max metric for rpcQueueTime, processing time.
     (Liang Xie via wang)
 
+    HADOOP-10143 replace WritableFactories's hashmap with ConcurrentHashMap
+    (Liang Xie via stack)
+
+    HADOOP-9652. Allow RawLocalFs#getFileLinkStatus to fill in the link owner
+    and mode if requested. (Andrew Wang via Colin Patrick McCabe)
+
   OPTIMIZATIONS
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
@@ -447,9 +453,6 @@ Release 2.4.0 - UNRELEASED
     HADOOP-9817. FileSystem#globStatus and FileContext#globStatus need to work
     with symlinks. (Colin Patrick McCabe via Andrew Wang)
 
-    HADOOP-9652.  RawLocalFs#getFileLinkStatus does not fill in the link owner
-    and mode.  (Andrew Wang via Colin Patrick McCabe)
-
     HADOOP-9875.  TestDoAsEffectiveUser can fail on JDK 7.  (Aaron T. Myers via
     Colin Patrick McCabe)
 
@@ -530,6 +533,9 @@ Release 2.4.0 - UNRELEASED
 
     HADOOP-10235. Hadoop tarball has 2 versions of stax-api JARs. (tucu)
 
+    HADOOP-10252. HttpServer can't start if hostname is not specified. (Jimmy
+    Xiang via atm)
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -550,6 +556,9 @@ Release 2.3.0 - UNRELEASED
     HADOOP-10132. RPC#stopProxy() should log the class of proxy when IllegalArgumentException 
     is encountered (Ted yu via umamahesh)
 
+    HADOOP-10248. Property name should be included in the exception where property value 
+    is null (Akira AJISAKA via umamahesh)
+
   OPTIMIZATIONS
 
     HADOOP-10142. Avoid groups lookup for unprivileged users such as "dr.who"
@@ -625,6 +634,9 @@ Release 2.3.0 - UNRELEASED
 
     HADOOP-10112. har file listing doesn't work with wild card. (brandonli)
 
+    HADOOP-10167. Mark hadoop-common source as UTF-8 in Maven pom files / refactoring
+    (Mikhail Antonov via cos)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -963,7 +963,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
         "Property name must not be null");
     Preconditions.checkArgument(
         value != null,
-        "Property value must not be null");
+        "The value of property " + name + " must not be null");
     DeprecationContext deprecations = deprecationContext.get();
     if (deprecations.getDeprecatedKeyMap().isEmpty()) {
       getProps();

+ 10 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java

@@ -16,8 +16,11 @@
  * limitations under the License.
  */
 
+
 package org.apache.hadoop.fs;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import java.io.BufferedOutputStream;
 import java.io.DataOutput;
 import java.io.File;
@@ -51,7 +54,13 @@ import org.apache.hadoop.util.StringUtils;
 public class RawLocalFileSystem extends FileSystem {
   static final URI NAME = URI.create("file:///");
   private Path workingDir;
-  private static final boolean useDeprecatedFileStatus = !Stat.isAvailable();
+  // Temporary workaround for HADOOP-9652.
+  private static boolean useDeprecatedFileStatus = true;
+
+  @VisibleForTesting
+  public static void useStatIfAvailable() {
+    useDeprecatedFileStatus = !Stat.isAvailable();
+  }
   
   public RawLocalFileSystem() {
     workingDir = getInitialWorkingDirectory();

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java

@@ -455,7 +455,7 @@ public class HttpServer implements FilterContainer {
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf, AccessControlList adminsAcl, 
       Connector connector, String[] pathSpecs) throws IOException {
-    this(new Builder().setName(name)
+    this(new Builder().setName(name).hostName(bindAddress)
         .addEndpoint(URI.create("http://" + bindAddress + ":" + port))
         .setFindPort(findPort).setConf(conf).setACL(adminsAcl)
         .setConnector(connector).setPathSpec(pathSpecs));

+ 6 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableFactories.java

@@ -22,25 +22,26 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.ReflectionUtils;
-import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 /** Factories for non-public writables.  Defining a factory permits {@link
  * ObjectWritable} to be able to construct instances of non-public classes. */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class WritableFactories {
-  private static final HashMap<Class, WritableFactory> CLASS_TO_FACTORY =
-    new HashMap<Class, WritableFactory>();
+  private static final Map<Class, WritableFactory> CLASS_TO_FACTORY =
+    new ConcurrentHashMap<Class, WritableFactory>();
 
   private WritableFactories() {}                  // singleton
 
   /** Define a factory for a class. */
-  public static synchronized void setFactory(Class c, WritableFactory factory) {
+  public static void setFactory(Class c, WritableFactory factory) {
     CLASS_TO_FACTORY.put(c, factory);
   }
 
   /** Define a factory for a class. */
-  public static synchronized WritableFactory getFactory(Class c) {
+  public static WritableFactory getFactory(Class c) {
     return CLASS_TO_FACTORY.get(c);
   }
 

+ 3 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java

@@ -1183,6 +1183,8 @@ public class TestConfiguration extends TestCase {
       fail("Should throw an IllegalArgumentException exception ");
     } catch (Exception e) {
       assertTrue(e instanceof IllegalArgumentException);
+      assertEquals(e.getMessage(),
+          "The value of property testClassName must not be null");
     }
   }
 
@@ -1193,6 +1195,7 @@ public class TestConfiguration extends TestCase {
       fail("Should throw an IllegalArgumentException exception ");
     } catch (Exception e) {
       assertTrue(e instanceof IllegalArgumentException);
+      assertEquals(e.getMessage(), "Property name must not be null");
     }
   }
 

+ 5 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFS.java

@@ -38,6 +38,11 @@ import org.junit.Test;
  * Test symbolic links using LocalFs.
  */
 abstract public class TestSymlinkLocalFS extends SymlinkBaseTest {
+
+  // Workaround for HADOOP-9652
+  static {
+    RawLocalFileSystem.useStatIfAvailable();
+  }
   
   @Override
   protected String getScheme() {

+ 11 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java

@@ -524,6 +524,17 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     Assert.assertFalse(HttpServer.isInstrumentationAccessAllowed(context, request, response));
   }
 
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testOldConstructor() throws Exception {
+    HttpServer server = new HttpServer("test", "0.0.0.0", 0, false);
+    try {
+      server.start();
+    } finally {
+      server.stop();
+    }
+  }
+
   @Test public void testBindAddress() throws Exception {
     checkBindAddress("localhost", 0, false).stop();
     // hang onto this one for a bit more testing

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml

@@ -554,6 +554,9 @@
                     <delete file="${httpfs.tomcat.dist.dir}/conf/server.xml"/>
                     <copy file="${basedir}/src/main/tomcat/server.xml"
                           toDir="${httpfs.tomcat.dist.dir}/conf"/>
+                    <delete file="${httpfs.tomcat.dist.dir}/conf/ssl-server.xml"/>
+                    <copy file="${basedir}/src/main/tomcat/ssl-server.xml"
+                          toDir="${httpfs.tomcat.dist.dir}/conf"/>
                     <delete file="${httpfs.tomcat.dist.dir}/conf/logging.properties"/>
                     <copy file="${basedir}/src/main/tomcat/logging.properties"
                           toDir="${httpfs.tomcat.dist.dir}/conf"/>

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh

@@ -39,3 +39,15 @@
 # The hostname HttpFS server runs on
 #
 # export HTTPFS_HTTP_HOSTNAME=`hostname -f`
+
+# Indicates if HttpFS is using SSL
+#
+# export HTTPFS_SSL_ENABLED=false
+
+# The location of the SSL keystore if using SSL
+#
+# export HTTPFS_SSL_KEYSTORE_FILE=${HOME}/.keystore
+
+# The password of the SSL keystore if using SSL
+#
+# export HTTPFS_SSL_KEYSTORE_PASS=password

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java

@@ -243,7 +243,7 @@ public class HttpFSFileSystem extends FileSystem
     if (makeQualified) {
       path = makeQualified(path);
     }
-    final URL url = HttpFSUtils.createHttpURL(path, params);
+    final URL url = HttpFSUtils.createURL(path, params);
     return doAsRealUserIfNecessary(new Callable<HttpURLConnection>() {
       @Override
       public HttpURLConnection call() throws Exception {

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSKerberosAuthenticator.java

@@ -123,7 +123,7 @@ public class HttpFSKerberosAuthenticator extends KerberosAuthenticator {
     Map<String, String> params = new HashMap<String, String>();
     params.put(OP_PARAM, op.toString());
     params.put(RENEWER_PARAM,renewer);
-    URL url = HttpFSUtils.createHttpURL(new Path(fsURI), params);
+    URL url = HttpFSUtils.createURL(new Path(fsURI), params);
     AuthenticatedURL aUrl =
       new AuthenticatedURL(new HttpFSKerberosAuthenticator());
     try {
@@ -150,7 +150,7 @@ public class HttpFSKerberosAuthenticator extends KerberosAuthenticator {
     params.put(OP_PARAM,
                DelegationTokenOperation.RENEWDELEGATIONTOKEN.toString());
     params.put(TOKEN_PARAM, dToken.encodeToUrlString());
-    URL url = HttpFSUtils.createHttpURL(new Path(fsURI), params);
+    URL url = HttpFSUtils.createURL(new Path(fsURI), params);
     AuthenticatedURL aUrl =
       new AuthenticatedURL(new HttpFSKerberosAuthenticator());
     try {
@@ -172,7 +172,7 @@ public class HttpFSKerberosAuthenticator extends KerberosAuthenticator {
     params.put(OP_PARAM,
                DelegationTokenOperation.CANCELDELEGATIONTOKEN.toString());
     params.put(TOKEN_PARAM, dToken.encodeToUrlString());
-    URL url = HttpFSUtils.createHttpURL(new Path(fsURI), params);
+    URL url = HttpFSUtils.createURL(new Path(fsURI), params);
     AuthenticatedURL aUrl =
       new AuthenticatedURL(new HttpFSKerberosAuthenticator());
     try {

+ 7 - 3
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java

@@ -55,17 +55,21 @@ public class HttpFSUtils {
    *
    * @return a <code>URL</code> for the HttpFSServer server,
    *
-   * @throws IOException thrown if an IO error occurrs.
+   * @throws IOException thrown if an IO error occurs.
    */
-  static URL createHttpURL(Path path, Map<String, String> params)
+  static URL createURL(Path path, Map<String, String> params)
     throws IOException {
     URI uri = path.toUri();
     String realScheme;
     if (uri.getScheme().equalsIgnoreCase(HttpFSFileSystem.SCHEME)) {
       realScheme = "http";
+    } else if (uri.getScheme().equalsIgnoreCase(HttpsFSFileSystem.SCHEME)) {
+      realScheme = "https";
+
     } else {
       throw new IllegalArgumentException(MessageFormat.format(
-        "Invalid scheme [{0}] it should be 'webhdfs'", uri));
+        "Invalid scheme [{0}] it should be '" + HttpFSFileSystem.SCHEME + "' " +
+            "or '" + HttpsFSFileSystem.SCHEME + "'", uri));
     }
     StringBuilder sb = new StringBuilder();
     sb.append(realScheme).append("://").append(uri.getAuthority()).

+ 34 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpsFSFileSystem.java

@@ -0,0 +1,34 @@
+/**
+ * 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.http.client;
+
+/**
+ * HttpFSServer implementation of the FileSystemAccess FileSystem for SSL.
+ * <p/>
+ * This implementation allows a user to access HDFS over HTTPS via a
+ * HttpFSServer server.
+ */
+public class HttpsFSFileSystem extends HttpFSFileSystem {
+
+  public static final String SCHEME = "swebhdfs";
+
+  @Override
+  public String getScheme() {
+    return SCHEME;
+  }
+}

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebApp.java

@@ -94,11 +94,11 @@ public class HttpFSServerWebApp extends ServerWebApp {
    */
   @Override
   public void init() throws ServerException {
-    super.init();
     if (SERVER != null) {
       throw new RuntimeException("HttpFSServer server already initialized");
     }
     SERVER = this;
+    super.init();
     adminGroup = getConfig().get(getPrefixedName(CONF_ADMIN_GROUP), "admin");
     LOG.info("Connects to Namenode [{}]",
              get().get(FileSystemAccess.class).getFileSystemConfiguration().

+ 8 - 5
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/DelegationTokenIdentifier.java

@@ -29,30 +29,33 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdenti
 public class DelegationTokenIdentifier
   extends AbstractDelegationTokenIdentifier {
 
-  public static final Text KIND_NAME = WebHdfsFileSystem.TOKEN_KIND;
+  private Text kind = WebHdfsFileSystem.TOKEN_KIND;
 
-  public DelegationTokenIdentifier() {
+  public DelegationTokenIdentifier(Text kind) {
+    this.kind = kind;
   }
 
   /**
    * Create a new delegation token identifier
    *
+   * @param kind token kind
    * @param owner the effective username of the token owner
    * @param renewer the username of the renewer
    * @param realUser the real username of the token owner
    */
-  public DelegationTokenIdentifier(Text owner, Text renewer, Text realUser) {
+  public DelegationTokenIdentifier(Text kind, Text owner, Text renewer,
+      Text realUser) {
     super(owner, renewer, realUser);
+    this.kind = kind;
   }
 
-
   /**
    * Returns the kind, <code>TOKEN_KIND</code>.
    * @return returns <code>TOKEN_KIND</code>.
    */
   @Override
   public Text getKind() {
-    return KIND_NAME;
+    return kind;
   }
 
 }

+ 14 - 5
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/security/DelegationTokenManagerService.java

@@ -19,6 +19,8 @@ package org.apache.hadoop.lib.service.security;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.http.server.HttpFSServerWebApp;
+import org.apache.hadoop.hdfs.web.SWebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.lib.server.BaseService;
 import org.apache.hadoop.lib.server.ServerException;
@@ -55,6 +57,8 @@ public class DelegationTokenManagerService extends BaseService
 
   DelegationTokenSecretManager secretManager = null;
 
+  private Text tokenKind;
+
   public DelegationTokenManagerService() {
     super(PREFIX);
   }
@@ -70,7 +74,9 @@ public class DelegationTokenManagerService extends BaseService
     long updateInterval = getServiceConfig().getLong(UPDATE_INTERVAL, DAY);
     long maxLifetime = getServiceConfig().getLong(MAX_LIFETIME, 7 * DAY);
     long renewInterval = getServiceConfig().getLong(RENEW_INTERVAL, DAY);
-    secretManager = new DelegationTokenSecretManager(updateInterval,
+    tokenKind = (HttpFSServerWebApp.get().isSslEnabled())
+                ? SWebHdfsFileSystem.TOKEN_KIND : WebHdfsFileSystem.TOKEN_KIND;
+    secretManager = new DelegationTokenSecretManager(tokenKind, updateInterval,
                                                      maxLifetime,
                                                      renewInterval, HOUR);
     try {
@@ -122,7 +128,7 @@ public class DelegationTokenManagerService extends BaseService
       realUser = new Text(ugi.getRealUser().getUserName());
     }
     DelegationTokenIdentifier tokenIdentifier =
-      new DelegationTokenIdentifier(owner, new Text(renewer), realUser);
+      new DelegationTokenIdentifier(tokenKind, owner, new Text(renewer), realUser);
     Token<DelegationTokenIdentifier> token =
       new Token<DelegationTokenIdentifier>(tokenIdentifier, secretManager);
     try {
@@ -188,7 +194,7 @@ public class DelegationTokenManagerService extends BaseService
     throws DelegationTokenManagerException {
     ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
     DataInputStream dis = new DataInputStream(buf);
-    DelegationTokenIdentifier id = new DelegationTokenIdentifier();
+    DelegationTokenIdentifier id = new DelegationTokenIdentifier(tokenKind);
     try {
       id.readFields(dis);
       dis.close();
@@ -203,6 +209,8 @@ public class DelegationTokenManagerService extends BaseService
   private static class DelegationTokenSecretManager
     extends AbstractDelegationTokenSecretManager<DelegationTokenIdentifier> {
 
+    private Text tokenKind;
+
     /**
      * Create a secret manager
      *
@@ -215,17 +223,18 @@ public class DelegationTokenManagerService extends BaseService
      * scanned
      * for expired tokens
      */
-    public DelegationTokenSecretManager(long delegationKeyUpdateInterval,
+    public DelegationTokenSecretManager(Text tokenKind, long delegationKeyUpdateInterval,
                                         long delegationTokenMaxLifetime,
                                         long delegationTokenRenewInterval,
                                         long delegationTokenRemoverScanInterval) {
       super(delegationKeyUpdateInterval, delegationTokenMaxLifetime,
             delegationTokenRenewInterval, delegationTokenRemoverScanInterval);
+      this.tokenKind = tokenKind;
     }
 
     @Override
     public DelegationTokenIdentifier createIdentifier() {
-      return new DelegationTokenIdentifier();
+      return new DelegationTokenIdentifier(tokenKind);
     }
 
   }

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/ServerWebApp.java

@@ -44,6 +44,7 @@ public abstract class ServerWebApp extends Server implements ServletContextListe
   private static final String TEMP_DIR = ".temp.dir";
   private static final String HTTP_HOSTNAME = ".http.hostname";
   private static final String HTTP_PORT = ".http.port";
+  public static final String SSL_ENABLED = ".ssl.enabled";
 
   private static ThreadLocal<String> HOME_DIR_TL = new ThreadLocal<String>();
 
@@ -225,4 +226,12 @@ public abstract class ServerWebApp extends Server implements ServletContextListe
   public void setAuthority(InetSocketAddress authority) {
     this.authority = authority;
   }
+
+
+  /**
+   *
+   */
+  public boolean isSslEnabled() {
+    return Boolean.valueOf(System.getProperty(getName() + SSL_ENABLED, "false"));
+  }
 }

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

@@ -143,6 +143,27 @@ else
   print "Using   HTTPFS_HTTP_HOSTNAME: ${HTTPFS_HTTP_HOSTNAME}"
 fi
 
+if [ "${HTTPFS_SSL_ENABLED}" = "" ]; then
+  export HTTPFS_SSL_ENABLED="false"
+  print "Setting HTTPFS_SSL_ENABLED: ${HTTPFS_SSL_ENABLED}"
+else
+  print "Using   HTTPFS_SSL_ENABLED: ${HTTPFS_SSL_ENABLED}"
+fi
+
+if [ "${HTTPFS_SSL_KEYSTORE_FILE}" = "" ]; then
+  export HTTPFS_SSL_KEYSTORE_FILE=${HOME}/.keystore
+  print "Setting HTTPFS_SSL_KEYSTORE_FILE:     ${HTTPFS_SSL_KEYSTORE_FILE}"
+else
+  print "Using   HTTPFS_SSL_KEYSTORE_FILE:     ${HTTPFS_SSL_KEYSTORE_FILE}"
+fi
+
+if [ "${HTTPFS_SSL_KEYSTORE_PASS}" = "" ]; then
+  export HTTPFS_SSL_KEYSTORE_PASS=password
+  print "Setting HTTPFS_SSL_KEYSTORE_PASS:     ${HTTPFS_SSL_KEYSTORE_PASS}"
+else
+  print "Using   HTTPFS_SSL_KEYSTORE_PASS:     ${HTTPFS_SSL_KEYSTORE_PASS}"
+fi
+
 if [ "${CATALINA_BASE}" = "" ]; then
   export CATALINA_BASE=${HTTPFS_HOME}/share/hadoop/httpfs/tomcat
   print "Setting CATALINA_BASE:       ${CATALINA_BASE}"

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/sbin/httpfs.sh

@@ -43,6 +43,9 @@ catalina_opts="${catalina_opts} -Dhttpfs.temp.dir=${HTTPFS_TEMP}";
 catalina_opts="${catalina_opts} -Dhttpfs.admin.port=${HTTPFS_ADMIN_PORT}";
 catalina_opts="${catalina_opts} -Dhttpfs.http.port=${HTTPFS_HTTP_PORT}";
 catalina_opts="${catalina_opts} -Dhttpfs.http.hostname=${HTTPFS_HTTP_HOSTNAME}";
+catalina_opts="${catalina_opts} -Dhttpfs.ssl.enabled=${HTTPFS_SSL_ENABLED}";
+catalina_opts="${catalina_opts} -Dhttpfs.ssl.keystore.file=${HTTPFS_SSL_KEYSTORE_FILE}";
+catalina_opts="${catalina_opts} -Dhttpfs.ssl.keystore.pass=${HTTPFS_SSL_KEYSTORE_PASS}";
 
 print "Adding to CATALINA_OPTS:     ${catalina_opts}"
 

+ 135 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml

@@ -0,0 +1,135 @@
+<?xml version='1.0' encoding='utf-8'?>
+<!--
+
+   All Rights Reserved.
+
+  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.
+-->
+<!-- Note:  A "Server" is not itself a "Container", so you may not
+     define subcomponents such as "Valves" at this level.
+     Documentation at /docs/config/server.html
+ -->
+<Server port="${httpfs.admin.port}" shutdown="SHUTDOWN">
+
+  <!--APR library loader. Documentation at /docs/apr.html -->
+  <Listener className="org.apache.catalina.core.AprLifecycleListener"
+            SSLEngine="on"/>
+  <!--Initialize Jasper prior to webapps are loaded. Documentation at /docs/jasper-howto.html -->
+  <Listener className="org.apache.catalina.core.JasperListener"/>
+  <!-- Prevent memory leaks due to use of particular java/javax APIs-->
+  <Listener
+    className="org.apache.catalina.core.JreMemoryLeakPreventionListener"/>
+  <!-- JMX Support for the Tomcat server. Documentation at /docs/non-existent.html -->
+  <Listener className="org.apache.catalina.mbeans.ServerLifecycleListener"/>
+  <Listener
+    className="org.apache.catalina.mbeans.GlobalResourcesLifecycleListener"/>
+
+  <!-- Global JNDI resources
+       Documentation at /docs/jndi-resources-howto.html
+  -->
+  <GlobalNamingResources>
+    <!-- Editable user database that can also be used by
+         UserDatabaseRealm to authenticate users
+    -->
+    <Resource name="UserDatabase" auth="Container"
+              type="org.apache.catalina.UserDatabase"
+              description="User database that can be updated and saved"
+              factory="org.apache.catalina.users.MemoryUserDatabaseFactory"
+              pathname="conf/tomcat-users.xml"/>
+  </GlobalNamingResources>
+
+  <!-- A "Service" is a collection of one or more "Connectors" that share
+       a single "Container" Note:  A "Service" is not itself a "Container",
+       so you may not define subcomponents such as "Valves" at this level.
+       Documentation at /docs/config/service.html
+   -->
+  <Service name="Catalina">
+
+    <!--The connectors can use a shared executor, you can define one or more named thread pools-->
+    <!--
+    <Executor name="tomcatThreadPool" namePrefix="catalina-exec-"
+        maxThreads="150" minSpareThreads="4"/>
+    -->
+
+    <!-- Define a SSL HTTP/1.1 Connector on port 8443
+         This connector uses the JSSE configuration, when using APR, the
+         connector should be using the OpenSSL style configuration
+         described in the APR documentation -->
+    <Connector port="${httpfs.http.port}" protocol="HTTP/1.1" SSLEnabled="true"
+               maxThreads="150" scheme="https" secure="true"
+               clientAuth="false" sslProtocol="TLS"
+               keystoreFile="${httpfs.ssl.keystore.file}"
+               keystorePass="${httpfs.ssl.keystore.pass}"/>
+
+    <!-- Define an AJP 1.3 Connector on port 8009 -->
+
+
+    <!-- An Engine represents the entry point (within Catalina) that processes
+ every request.  The Engine implementation for Tomcat stand alone
+ analyzes the HTTP headers included with the request, and passes them
+ on to the appropriate Host (virtual host).
+ Documentation at /docs/config/engine.html -->
+
+    <!-- You should set jvmRoute to support load-balancing via AJP ie :
+    <Engine name="Catalina" defaultHost="localhost" jvmRoute="jvm1">
+    -->
+    <Engine name="Catalina" defaultHost="localhost">
+
+      <!--For clustering, please take a look at documentation at:
+          /docs/cluster-howto.html  (simple how to)
+          /docs/config/cluster.html (reference documentation) -->
+      <!--
+      <Cluster className="org.apache.catalina.ha.tcp.SimpleTcpCluster"/>
+      -->
+
+      <!-- The request dumper valve dumps useful debugging information about
+           the request and response data received and sent by Tomcat.
+           Documentation at: /docs/config/valve.html -->
+      <!--
+      <Valve className="org.apache.catalina.valves.RequestDumperValve"/>
+      -->
+
+      <!-- This Realm uses the UserDatabase configured in the global JNDI
+           resources under the key "UserDatabase".  Any edits
+           that are performed against this UserDatabase are immediately
+           available for use by the Realm.  -->
+      <Realm className="org.apache.catalina.realm.UserDatabaseRealm"
+             resourceName="UserDatabase"/>
+
+      <!-- Define the default virtual host
+           Note: XML Schema validation will not work with Xerces 2.2.
+       -->
+      <Host name="localhost" appBase="webapps"
+            unpackWARs="true" autoDeploy="true"
+            xmlValidation="false" xmlNamespaceAware="false">
+
+        <!-- SingleSignOn valve, share authentication between web applications
+             Documentation at: /docs/config/valve.html -->
+        <!--
+        <Valve className="org.apache.catalina.authenticator.SingleSignOn" />
+        -->
+
+        <!-- Access log processes all example.
+             Documentation at: /docs/config/valve.html -->
+        <!--
+        <Valve className="org.apache.catalina.valves.AccessLogValve" directory="logs"
+               prefix="localhost_access_log." suffix=".txt" pattern="common" resolveHosts="false"/>
+        -->
+
+      </Host>
+    </Engine>
+  </Service>
+</Server>

+ 42 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/ServerSetup.apt.vm

@@ -118,4 +118,46 @@ Transfer-Encoding: chunked
   HttpFS supports the following {{{./httpfs-default.html}configuration properties}}
   in the HttpFS's <<<conf/httpfs-site.xml>>> configuration file.
 
+* HttpFS over HTTPS (SSL)
+
+  To configure HttpFS to work over SSL edit the {{httpfs-env.sh}} script in the
+  configuration directory setting the {{HTTPFS_SSL_ENABLED}} to {{true}}.
+
+  In addition, the following 2 properties may be defined (shown with default
+  values):
+
+    * HTTPFS_SSL_KEYSTORE_FILE=${HOME}/.keystore
+
+    * HTTPFS_SSL_KEYSTORE_PASS=password
+
+  In the HttpFS <<<tomcat/conf>>> directory, replace the <<<server.xml>>> file
+  with the  <<<ssl-server.xml>>> file.
+
+
+  You need to create an SSL certificate for the HttpFS server. As the
+  <<<httpfs>>> Unix user, using the Java <<<keytool>>> command to create the
+  SSL certificate:
+
++---+
+$ keytool -genkey -alias tomcat -keyalg RSA
++---+
+
+  You will be asked a series of questions in an interactive prompt.  It will
+  create the keystore file, which will be named <<.keystore>> and located in the
+  <<<httpfs>>> user home directory.
+
+  The password you enter for "keystore password" must match the  value of the
+  <<<HTTPFS_SSL_KEYSTORE_PASS>>> environment variable set in the
+  <<<httpfs-env.sh>>> script in the configuration directory.
+
+  The answer to "What is your first and last name?" (i.e. "CN") must be the
+  hostname of the machine where the HttpFS Server will be running.
+
+  Start HttpFS. It should work over HTTPS.
+
+  Using the Hadoop <<<FileSystem>>> API or the Hadoop FS shell, use the
+  <<<swebhdfs://>>> scheme. Make sure the JVM is picking up the truststore
+  containing the public key of the SSL certificate if using a self-signed
+  certificate.
+
   \[ {{{./index.html}Go Back}} \]

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java

@@ -116,10 +116,14 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     return HttpFSFileSystem.class;
   }
 
+  protected String getScheme() {
+    return "webhdfs";
+  }
+
   protected FileSystem getHttpFSFileSystem() throws Exception {
     Configuration conf = new Configuration();
     conf.set("fs.webhdfs.impl", getFileSystemClass().getName());
-    URI uri = new URI("webhdfs://" +
+    URI uri = new URI(getScheme() + "://" +
                       TestJettyHelper.getJettyURL().toURI().getAuthority());
     return FileSystem.get(uri, conf);
   }
@@ -127,7 +131,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
   protected void testGet() throws Exception {
     FileSystem fs = getHttpFSFileSystem();
     Assert.assertNotNull(fs);
-    URI uri = new URI("webhdfs://" +
+    URI uri = new URI(getScheme() + "://" +
                       TestJettyHelper.getJettyURL().toURI().getAuthority());
     Assert.assertEquals(fs.getUri(), uri);
     fs.close();

+ 99 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithSWebhdfsFileSystem.java

@@ -0,0 +1,99 @@
+/**
+ * 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.http.client;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.web.SWebHdfsFileSystem;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.TestJettyHelper;
+import org.junit.AfterClass;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.File;
+import java.net.URI;
+import java.net.URL;
+import java.util.UUID;
+
+@RunWith(value = Parameterized.class)
+public class TestHttpFSFWithSWebhdfsFileSystem
+  extends TestHttpFSWithHttpFSFileSystem {
+  private static String classpathDir;
+  private static final String BASEDIR = System.getProperty("test.build.dir",
+      "target/test-dir") + "/" + UUID.randomUUID();
+
+  private static Configuration sslConf;
+
+  {
+    URL url = Thread.currentThread().getContextClassLoader().
+        getResource("classutils.txt");
+    classpathDir = url.toExternalForm();
+    if (classpathDir.startsWith("file:")) {
+      classpathDir = classpathDir.substring("file:".length());
+      classpathDir = classpathDir.substring(0,
+          classpathDir.length() - "/classutils.txt".length());
+    } else {
+      throw new RuntimeException("Cannot find test classes dir");
+    }
+    File base = new File(BASEDIR);
+    FileUtil.fullyDelete(base);
+    base.mkdirs();
+    String keyStoreDir = new File(BASEDIR).getAbsolutePath();
+    try {
+      sslConf = new Configuration();
+      KeyStoreTestUtil.setupSSLConfig(keyStoreDir, classpathDir, sslConf, false);
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+    jettyTestHelper = new TestJettyHelper("jks", keyStoreDir + "/serverKS.jks",
+        "serverP");
+  }
+
+  @AfterClass
+  public static void cleanUp() {
+    new File(classpathDir, "ssl-client.xml").delete();
+    new File(classpathDir, "ssl-server.xml").delete();
+  }
+
+  public TestHttpFSFWithSWebhdfsFileSystem(Operation operation) {
+    super(operation);
+  }
+
+  @Override
+  protected Class getFileSystemClass() {
+    return SWebHdfsFileSystem.class;
+  }
+
+  @Override
+  protected String getScheme() {
+    return "swebhdfs";
+  }
+
+  @Override
+  protected FileSystem getHttpFSFileSystem() throws Exception {
+    Configuration conf = new Configuration(sslConf);
+    conf.set("fs.swebhdfs.impl", getFileSystemClass().getName());
+    URI uri = new URI("swebhdfs://" +
+        TestJettyHelper.getJettyURL().toURI().getAuthority());
+    return FileSystem.get(uri, conf);
+  }
+
+}

+ 27 - 5
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSKerberosAuthenticationHandler.java

@@ -22,9 +22,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
 import org.apache.hadoop.fs.http.client.HttpFSKerberosAuthenticator;
 import org.apache.hadoop.fs.http.client.HttpFSKerberosAuthenticator.DelegationTokenOperation;
+import org.apache.hadoop.hdfs.web.SWebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.lib.service.DelegationTokenIdentifier;
 import org.apache.hadoop.lib.service.DelegationTokenManager;
 import org.apache.hadoop.lib.service.DelegationTokenManagerException;
+import org.apache.hadoop.lib.servlet.ServerWebApp;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
@@ -51,7 +55,24 @@ public class TestHttpFSKerberosAuthenticationHandler extends HFSTestCase {
 
   @Test
   @TestDir
-  public void testManagementOperations() throws Exception {
+  public void testManagementOperationsWebHdfsFileSystem() throws Exception {
+    testManagementOperations(WebHdfsFileSystem.TOKEN_KIND);
+  }
+
+  @Test
+  @TestDir
+  public void testManagementOperationsSWebHdfsFileSystem() throws Exception {
+    try {
+      System.setProperty(HttpFSServerWebApp.NAME +
+          ServerWebApp.SSL_ENABLED, "true");
+      testManagementOperations(SWebHdfsFileSystem.TOKEN_KIND);
+    } finally {
+      System.getProperties().remove(HttpFSServerWebApp.NAME +
+          ServerWebApp.SSL_ENABLED);
+    }
+  }
+
+  private void testManagementOperations(Text expectedTokenKind) throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
 
     Configuration httpfsConf = new Configuration(false);
@@ -67,8 +88,8 @@ public class TestHttpFSKerberosAuthenticationHandler extends HFSTestCase {
 
       testNonManagementOperation(handler);
       testManagementOperationErrors(handler);
-      testGetToken(handler, null);
-      testGetToken(handler, "foo");
+      testGetToken(handler, null, expectedTokenKind);
+      testGetToken(handler, "foo", expectedTokenKind);
       testCancelToken(handler);
       testRenewToken(handler);
 
@@ -112,8 +133,8 @@ public class TestHttpFSKerberosAuthenticationHandler extends HFSTestCase {
       Mockito.contains("requires SPNEGO"));
   }
 
-  private void testGetToken(AuthenticationHandler handler, String renewer)
-    throws Exception {
+  private void testGetToken(AuthenticationHandler handler, String renewer,
+      Text expectedTokenKind) throws Exception {
     DelegationTokenOperation op = DelegationTokenOperation.GETDELEGATIONTOKEN;
     HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
     HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
@@ -154,6 +175,7 @@ public class TestHttpFSKerberosAuthenticationHandler extends HFSTestCase {
     Token<DelegationTokenIdentifier> dt = new Token<DelegationTokenIdentifier>();
     dt.decodeFromUrlString(tokenStr);
     HttpFSServerWebApp.get().get(DelegationTokenManager.class).verifyToken(dt);
+    Assert.assertEquals(expectedTokenKind, dt.getKind());
   }
 
   private void testCancelToken(AuthenticationHandler handler)

+ 9 - 3
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/security/TestDelegationTokenManagerService.java

@@ -23,6 +23,9 @@ import org.apache.hadoop.fs.http.server.HttpFSServerWebApp;
 import org.apache.hadoop.lib.server.Server;
 import org.apache.hadoop.lib.service.DelegationTokenManager;
 import org.apache.hadoop.lib.service.DelegationTokenManagerException;
+import org.apache.hadoop.lib.service.hadoop.FileSystemAccessService;
+import org.apache.hadoop.lib.service.instrumentation.InstrumentationService;
+import org.apache.hadoop.lib.service.scheduler.SchedulerService;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.HTestCase;
@@ -43,9 +46,12 @@ public class TestDelegationTokenManagerService extends HTestCase {
   public void service() throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     Configuration conf = new Configuration(false);
-    conf.set("server.services", StringUtils.join(",",
-      Arrays.asList(DelegationTokenManagerService.class.getName())));
-    Server server = new Server("server", dir, dir, dir, dir, conf);
+    conf.set("httpfs.services", StringUtils.join(",",
+      Arrays.asList(InstrumentationService.class.getName(),
+          SchedulerService.class.getName(),
+          FileSystemAccessService.class.getName(),
+          DelegationTokenManagerService.class.getName())));
+    Server server = new HttpFSServerWebApp(dir, dir, dir, dir, conf);
     server.init();
     DelegationTokenManager tm = server.get(DelegationTokenManager.class);
     Assert.assertNotNull(tm);

+ 43 - 14
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java

@@ -28,31 +28,46 @@ import org.junit.Test;
 import org.junit.rules.MethodRule;
 import org.junit.runners.model.FrameworkMethod;
 import org.junit.runners.model.Statement;
+import org.mortbay.jetty.Connector;
 import org.mortbay.jetty.Server;
+import org.mortbay.jetty.security.SslSocketConnector;
 
 public class TestJettyHelper implements MethodRule {
+  private boolean ssl;
+  private String keyStoreType;
+  private String keyStore;
+  private String keyStorePassword;
+  private Server server;
 
-  @Test
-  public void dummy() {
+  public TestJettyHelper() {
+    this.ssl = false;
   }
 
-  private static ThreadLocal<Server> TEST_SERVLET_TL = new InheritableThreadLocal<Server>();
+  public TestJettyHelper(String keyStoreType, String keyStore,
+      String keyStorePassword) {
+    ssl = true;
+    this.keyStoreType = keyStoreType;
+    this.keyStore = keyStore;
+    this.keyStorePassword = keyStorePassword;
+  }
+
+  private static ThreadLocal<TestJettyHelper> TEST_JETTY_TL =
+      new InheritableThreadLocal<TestJettyHelper>();
 
   @Override
   public Statement apply(final Statement statement, final FrameworkMethod frameworkMethod, final Object o) {
     return new Statement() {
       @Override
       public void evaluate() throws Throwable {
-        Server server = null;
         TestJetty testJetty = frameworkMethod.getAnnotation(TestJetty.class);
         if (testJetty != null) {
           server = createJettyServer();
         }
         try {
-          TEST_SERVLET_TL.set(server);
+          TEST_JETTY_TL.set(TestJettyHelper.this);
           statement.evaluate();
         } finally {
-          TEST_SERVLET_TL.remove();
+          TEST_JETTY_TL.remove();
           if (server != null && server.isRunning()) {
             try {
               server.stop();
@@ -73,8 +88,19 @@ public class TestJettyHelper implements MethodRule {
       int port = ss.getLocalPort();
       ss.close();
       Server server = new Server(0);
-      server.getConnectors()[0].setHost(host);
-      server.getConnectors()[0].setPort(port);
+      if (!ssl) {
+        server.getConnectors()[0].setHost(host);
+        server.getConnectors()[0].setPort(port);
+      } else {
+        SslSocketConnector c = new SslSocketConnector();
+        c.setHost(host);
+        c.setPort(port);
+        c.setNeedClientAuth(false);
+        c.setKeystore(keyStore);
+        c.setKeystoreType(keyStoreType);
+        c.setKeyPassword(keyStorePassword);
+        server.setConnectors(new Connector[] {c});
+      }
       return server;
     } catch (Exception ex) {
       throw new RuntimeException("Could not start embedded servlet container, " + ex.getMessage(), ex);
@@ -109,11 +135,11 @@ public class TestJettyHelper implements MethodRule {
    * @return a Jetty server ready to be configured and the started.
    */
   public static Server getJettyServer() {
-    Server server = TEST_SERVLET_TL.get();
-    if (server == null) {
+    TestJettyHelper helper = TEST_JETTY_TL.get();
+    if (helper == null || helper.server == null) {
       throw new IllegalStateException("This test does not use @TestJetty");
     }
-    return server;
+    return helper.server;
   }
 
   /**
@@ -123,12 +149,15 @@ public class TestJettyHelper implements MethodRule {
    * @return the base URL (SCHEMA://HOST:PORT) of the test Jetty server.
    */
   public static URL getJettyURL() {
-    Server server = TEST_SERVLET_TL.get();
-    if (server == null) {
+    TestJettyHelper helper = TEST_JETTY_TL.get();
+    if (helper == null || helper.server == null) {
       throw new IllegalStateException("This test does not use @TestJetty");
     }
     try {
-      return new URL("http://" + server.getConnectors()[0].getHost() + ":" + server.getConnectors()[0].getPort());
+      String scheme = (helper.ssl) ? "https" : "http";
+      return new URL(scheme + "://" +
+          helper.server.getConnectors()[0].getHost() + ":" +
+          helper.server.getConnectors()[0].getPort());
     } catch (MalformedURLException ex) {
       throw new RuntimeException("It should never happen, " + ex.getMessage(), ex);
     }

+ 238 - 204
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -120,94 +120,9 @@ Trunk (Unreleased)
     HDFS-5041. Add the time of last heartbeat to dead server Web UI (Shinichi
     Yamashita via brandonli)
 
-    HDFS-5049.  Add JNI mlock support.  (Andrew Wang via Colin Patrick McCabe)
-
-    HDFS-5051.  Propagate cache status information from the DataNode to the
-    NameNode  (Andrew Wang via Colin Patrick McCabe)
-
-    HDFS-5052.  Add cacheRequest/uncacheRequest support to NameNode.
-    (contributed by Colin Patrick McCabe)
-
-    HDFS-5050.  Add DataNode support for mlock and munlock
-    (Andrew Wang via Colin Patrick McCabe)
-
-    HDFS-5141. Add cache status information to datanode heartbeat.
-    (Contributed by Andrew Wang)
-
-    HDFS-5121. Add RPCs for creating and manipulating cache pools.
-    (Contributed by Colin Patrick McCabe)
-
-    HDFS-5163. Miscellaneous cache pool RPC fixes.  (Contributed by Colin
-    Patrick McCabe)
-
-    HDFS-5120. Add command-line support for manipulating cache pools.
-    (Contributed by Colin Patrick McCabe)
-
-    HDFS-5158. Add command-line support for manipulating cache directives.
-    (Contributed by Colin Patrick McCabe)
-
-    HDFS-5053. NameNode should invoke DataNode APIs to coordinate caching.
-    (Andrew Wang)
-
-    HDFS-5197. Document dfs.cachereport.intervalMsec in hdfs-default.xml.
-    (cnauroth)
-
-    HDFS-5213. Separate PathBasedCacheEntry and PathBasedCacheDirectiveWithId.
-    (Contributed by Colin Patrick McCabe)
-
-    HDFS-5236. Change PathBasedCacheDirective APIs to be a single value
-    rather than batch. (Contributed by Andrew Wang)
-
-    HDFS-5191. Revisit zero-copy API in FSDataInputStream to make it more
-    intuitive.  (Contributed by Colin Patrick McCabe)
-
-    HDFS-5119. Persist CacheManager state in the edit log.
-    (Contributed by Andrew Wang)
-
-    HDFS-5190. Move cache pool related CLI commands to CacheAdmin.
-    (Contributed by Andrew Wang)
-
-    HDFS-5304. Expose if a block replica is cached in getFileBlockLocations.
-    (Contributed by Andrew Wang)
-
-    HDFS-5224. Refactor PathBasedCache* methods to use a Path rather than a
-    String. (cnauroth)
-
-    HDFS-5358. Add replication field to PathBasedCacheDirective.
-    (Contributed by Colin Patrick McCabe)
-
-    HDFS-5359. Allow LightWeightGSet#Iterator to remove elements.
-    (Contributed by Colin Patrick McCabe)
-
-    HDFS-5096. Automatically cache new data added to a cached path.
-    (Contributed by Colin Patrick McCabe)
-
-    HDFS-5378. In CacheReport, don't send genstamp and length on the wire
-    (Contributed by Colin Patrick McCabe)
-
-    HDFS-5386. Add feature documentation for datanode caching.
-    (Colin Patrick McCabe via cnauroth)
-
-    HDFS-5326. add modifyDirective to cacheAdmin.  (cmccabe)
-
-    HDFS-5450. Better API for getting the cached blocks locations. (wang)
-
-    HDFS-5485. Add command-line support for modifyDirective. (cmccabe)
-
-    HDFS-5366. recaching improvements (cmccabe)
-
-    HDFS-5511. improve CacheManipulator interface to allow better unit testing
-    (cmccabe)
-
-    HDFS-5451. Add byte and file statistics to PathBasedCacheEntry.
-    (Colin Patrick McCabe via Andrew Wang)
-
     HDFS-5531. Combine the getNsQuota() and getDsQuota() methods in INode.
     (szetszwo)
 
-    HDFS-5473. Consistent naming of user-visible caching classes and methods
-    (cmccabe)
-
     HDFS-5285. Flatten INodeFile hierarchy: Replace INodeFileUnderConstruction
     and INodeFileUnderConstructionWithSnapshot with FileUnderContructionFeature.
     (jing9 via szetszwo)
@@ -215,15 +130,8 @@ Trunk (Unreleased)
     HDFS-5286. Flatten INodeDirectory hierarchy: Replace INodeDirectoryWithQuota
     with DirectoryWithQuotaFeature.  (szetszwo)
 
-    HDFS-5556. Add some more NameNode cache statistics, cache pool stats
-    (cmccabe)
-
     HDFS-5537. Remove FileWithSnapshot interface.  (jing9 via szetszwo)
 
-    HDFS-5430. Support TTL on CacheDirectives. (wang)
-
-    HDFS-5630. Hook up cache directive and pool usage statistics. (wang)
-
     HDFS-5554. Flatten INodeFile hierarchy: Replace INodeFileWithSnapshot with
     FileWithSnapshotFeature.  (jing9 via szetszwo)
 
@@ -234,26 +142,15 @@ Trunk (Unreleased)
     INodeDirectoryWithSnapshot with DirectoryWithSnapshotFeature.
     (jing9 via szetszwo)
 
-    HDFS-5431. Support cachepool-based limit management in path-based caching
-    (awang via cmccabe)
-
-    HDFS-5636. Enforce a max TTL per cache pool. (awang via cmccabe)
-
-    HDFS-5651. Remove dfs.namenode.caching.enabled and improve CRM locking.
-    (cmccabe via wang)
-
     HDFS-5715. Use Snapshot ID to indicate the corresponding Snapshot for a
     FileDiff/DirectoryDiff. (jing9)
 
     HDFS-5721. sharedEditsImage in Namenode#initializeSharedEdits() should be 
     closed before method returns. (Ted Yu via junping_du)
 
-  OPTIMIZATIONS
-
-    HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
+    HDFS-5138. Support HDFS upgrade in HA. (atm via todd)
 
-    HDFS-5665. Remove the unnecessary writeLock while initializing CacheManager
-    in FsNameSystem Ctor. (Uma Maheswara Rao G via Andrew Wang)
+  OPTIMIZATIONS
 
   BUG FIXES
 
@@ -372,110 +269,12 @@ Trunk (Unreleased)
     HDFS-4366. Block Replication Policy Implementation May Skip Higher-Priority
     Blocks for Lower-Priority Blocks (Derek Dagit via kihwal)
 
-    HDFS-5169. hdfs.c: translateZCRException: null pointer deref when
-    translating some exceptions.  (Contributed by Colin Patrick McCabe)
-
-    HDFS-5198. NameNodeRpcServer must not send back DNA_FINALIZE in reply to a
-    cache report.  (Contributed by Colin Patrick McCabe)
-
-    HDFS-5195. Prevent passing null pointer to mlock and munlock. (cnauroth)
-
-    HDFS-5201. NativeIO: consolidate getrlimit into NativeIO#getMemlockLimit
-    (Contributed by Colin Patrick McCabe)
-
-    HDFS-5210. Fix some failing unit tests on HDFS-4949 branch.
-    (Contributed by Andrew Wang)
-
-    HDFS-5266. ElasticByteBufferPool#Key does not implement equals. (cnauroth)
-
-    HDFS-5309. Fix failing caching unit tests. (Andrew Wang)
-
-    HDFS-5314. Do not expose CachePool type in AddCachePoolOp (Colin Patrick
-    McCabe)
-
-    HDFS-5348. Fix error message when dfs.datanode.max.locked.memory is
-    improperly configured. (Colin Patrick McCabe)
-
-    HDFS-5373. hdfs cacheadmin -addDirective short usage does not mention
-    -replication parameter. (cnauroth)
-
-    HDFS-5383. fix broken caching unit tests. (Andrew Wang)
-
-    HDFS-5388. Loading fsimage fails to find cache pools during namenode
-    startup.  (Chris Nauroth via Colin Patrick McCabe)
-
-    HDFS-5203. Concurrent clients that add a cache directive on the same path
-    may prematurely uncache from each other.  (Chris Nauroth via Colin Patrick
-    McCabe)
-
-    HDFS-5385. Caching RPCs are AtMostOnce, but do not persist client ID and
-    call ID to edit log.  (Chris Nauroth via Colin Patrick McCabe)
-
-    HDFS-5404. Resolve regressions in Windows compatibility on HDFS-4949
-    branch. (Chris Nauroth via Andrew Wang)
-
-    HDFS-5405. Fix possible RetryCache hang for caching RPC handlers in
-    FSNamesystem. (wang)
-
-    HDFS-5419. Fixup test-patch.sh warnings on HDFS-4949 branch. (wang)
-
-    HDFS-5468. CacheAdmin help command does not recognize commands  (Stephen
-    Chu via Colin Patrick McCabe)
-
-    HDFS-5394. Fix race conditions in DN caching and uncaching (cmccabe)
-
-    HDFS-5482. DistributedFileSystem#listPathBasedCacheDirectives must support
-    relative paths. (Colin Patrick McCabe via cnauroth)
-
-    HDFS-5320. Add datanode caching metrics. (wang)
-
-    HDFS-5520. loading cache path directives from edit log doesn't update
-    nextEntryId (cmccabe)
-
-    HDFS-5512. CacheAdmin -listPools fails with NPE when user lacks permissions
-    to view all pools (wang via cmccabe)
-
-    HDFS-5513. CacheAdmin commands fail when using . as the path. (wang)
-
-    HDFS-5543. Fix narrow race condition in TestPathBasedCacheRequests
-    (cmccabe)
-
-    HDFS-5565. CacheAdmin help should match against non-dashed commands
-    (wang via cmccabe)
-
-    HDFS-5562. TestCacheDirectives and TestFsDatasetCache should stub out
-    native mlock. (Colin McCabe and Akira Ajisaka via wang)
-
-    HDFS-5555. CacheAdmin commands fail when first listed NameNode is in
-    Standby (jxiang via cmccabe)
-
-    HDFS-5626. dfsadmin -report shows incorrect cache values. (cmccabe)
-
-    HDFS-5679. TestCacheDirectives should handle the case where native code
-    is not available. (wang)
-
-    HDFS-5701. Fix the CacheAdmin -addPool -maxTtl option name.
-    (Stephen Chu via wang)
-
-    HDFS-5708. The CacheManager throws a NPE in the DataNode logs when
-    processing cache reports that refer to a block not known to the
-    BlockManager. (cmccabe via wang)
-
-    HDFS-5659. dfsadmin -report doesn't output cache information properly.
-    (wang)
-
     HDFS-5705. TestSecondaryNameNodeUpgrade#testChangeNsIDFails may fail due
     to ConcurrentModificationException. (Ted Yu via brandonli)
 
     HDFS-5719. FSImage#doRollback() should close prevState before return
     (Ted Yu via brandonli)
 
-    HDFS-5589. Namenode loops caching and uncaching when data should be
-    uncached (awang via cmccabe)
-
-    HDFS-5724. modifyCacheDirective logging audit log command wrongly as
-    addCacheDirective (Uma Maheswara Rao G via Colin Patrick McCabe)
-
     HDFS-5726. Fix compilation error in AbstractINodeDiff for JDK7. (jing9)
 
     HDFS-5768. Consolidate the serialization code in DelegationTokenSecretManager 
@@ -524,6 +323,10 @@ Release 2.4.0 - UNRELEASED
     HDFS-5784. reserve space in edit log header and fsimage header for feature
     flag section (cmccabe)
 
+    HDFS-5703. Add support for HTTPS and swebhdfs to HttpFS. (tucu)
+
+    HDFS-4949. Centralized cache management in HDFS. (wang and cmccabe)
+
   IMPROVEMENTS
 
     HDFS-5267. Remove volatile from LightWeightHashSet. (Junping Du via llu)
@@ -695,6 +498,12 @@ Release 2.4.0 - UNRELEASED
 
     HDFS-5704. Change OP_UPDATE_BLOCKS with a new OP_ADD_BLOCK. (jing9)
 
+    HDFS-5434. Change block placement policy constructors from package private
+    to protected. (Buddy Taylor via Arpit Agarwal)
+
+    HDFS-5788. listLocatedStatus response can be very large. (Nathan Roberts
+    via kihwal)
+
   OPTIMIZATIONS
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
@@ -704,6 +513,9 @@ Release 2.4.0 - UNRELEASED
 
     HDFS-5681. renewLease should not hold fsn write lock. (daryn via Kihwal)
 
+    HDFS-5241. Provide alternate queuing audit logger to reduce logging
+    contention (daryn)
+
   BUG FIXES
 
     HDFS-5034.  Remove debug prints from GetFileLinkInfo (Andrew Wang via Colin
@@ -775,6 +587,18 @@ Release 2.4.0 - UNRELEASED
 
     HDFS-5777. Update LayoutVersion for the new editlog op OP_ADD_BLOCK. (jing9)
 
+    HDFS-5800. Fix a typo in DFSClient.renewLease().  (Kousuke Saruta
+    via szetszwo)
+
+    HDFS-5748. Too much information shown in the dfs health page.
+    (Haohui Mai via brandonli)
+
+    HDFS-5806. balancer should set SoTimeout to avoid indefinite hangs.
+    (Nathan Roberts via Andrew Wang).
+
+    HDFS-5728. Block recovery will fail if the metafile does not have crc 
+    for all chunks of the block (Vinay via kihwal)
+
   BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS
 
     HDFS-4985. Add storage type to the protocol and expose it in block report
@@ -908,6 +732,211 @@ Release 2.4.0 - UNRELEASED
 
     HDFS-5667. Include DatanodeStorage in StorageReport. (Arpit Agarwal)
 
+  BREAKDOWN OF HDFS-4949 SUBTASKS AND RELATED JIRAS
+
+    HDFS-5049.  Add JNI mlock support.  (Andrew Wang via Colin Patrick McCabe)
+
+    HDFS-5051.  Propagate cache status information from the DataNode to the
+    NameNode  (Andrew Wang via Colin Patrick McCabe)
+
+    HDFS-5052. Add cacheRequest/uncacheRequest support to NameNode.
+    (Contributed by Colin Patrick McCabe.)
+
+    HDFS-5050.  Add DataNode support for mlock and munlock  (contributed by
+    Andrew Wang)
+
+    HDFS-5141. Add cache status information to datanode heartbeat. (Contributed
+    by Andrew Wang)
+
+    HDFS-5121.  Add RPCs for creating and manipulating cache pools.
+    (Contributed by Colin Patrick McCabe)
+
+    HDFS-5163. Miscellaneous cache pool RPC fixes (Contributed by Colin Patrick
+    McCabe)
+
+    HDFS-5169. hdfs.c: translateZCRException: null pointer deref when
+    translating some exceptions (Contributed by Colin Patrick McCabe)
+
+    HDFS-5120. Add command-line support for manipulating cache pools. (cmccabe)
+
+    HDFS-5158. Add command-line support for manipulating cache directives.
+    (cmccabe)
+
+    HDFS-5198. NameNodeRpcServer must not send back DNA_FINALIZE in reply to a
+    cache report. (cmccabe)
+
+    HDFS-5195. Prevent passing null pointer to mlock and munlock. Contributed
+    by Chris Nauroth.
+
+    HDFS-5053. NameNode should invoke DataNode APIs to coordinate caching.
+    (Andrew Wang)
+
+    HDFS-5201. NativeIO: consolidate getrlimit into NativeIO#getMemlockLimit.
+    (Contributed by Colin Patrick McCabe)
+
+    HDFS-5197. Document dfs.cachereport.intervalMsec in hdfs-default.xml.
+    Contributed by Chris Nauroth.
+
+    HDFS-5210. Fix some failing unit tests on HDFS-4949 branch. (Contributed by
+    Andrew Wang)
+
+    HDFS-5213. Separate PathBasedCacheEntry and PathBasedCacheDirectiveWithId.
+    Contributed by Colin Patrick McCabe.
+
+    HDFS-5236. Change PathBasedCacheDirective APIs to be a single value rather
+    than batch. (Contributed by Andrew Wang)
+
+    HDFS-5119. Persist CacheManager state in the edit log. (Contributed by
+    Andrew Wang)
+
+    HDFS-5190. Move cache pool related CLI commands to CacheAdmin. (Contributed
+    by Andrew Wang)
+
+    HDFS-5309. Fix failing caching unit tests. (Andrew Wang)
+
+    HDFS-5314.  Do not expose CachePool type in AddCachePoolOp (Colin Patrick
+    McCabe)
+
+    HDFS-5304. Expose if a block replica is cached in getFileBlockLocations.
+    (Contributed by Andrew Wang)
+
+    HDFS-5224. Refactor PathBasedCache* methods to use a Path rather than a
+    String. Contributed by Chris Nauroth.
+
+    HDFS-5348. Fix error message when dfs.datanode.max.locked.memory is
+    improperly configured. (Contributed by Colin Patrick McCabe)
+
+    HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only (cmccabe)
+
+    HDFS-5358. Add replication field to PathBasedCacheDirective. (Contributed
+    by Colin Patrick McCabe)
+
+    HDFS-5359. Allow LightWeightGSet#Iterator to remove elements. (Contributed
+    by Colin Patrick McCabe)
+
+    HDFS-5373. hdfs cacheadmin -addDirective short usage does not mention
+    -replication parameter. Contributed by Chris Nauroth.
+
+    HDFS-5096. Automatically cache new data added to a cached path (contributed
+    by Colin Patrick McCabe)
+
+    HDFS-5383. fix broken caching unit tests (Andrew Wang)
+
+    HDFS-5388. Loading fsimage fails to find cache pools during namenode
+    startup (Chris Nauroth via Colin Patrick McCabe)
+
+    HDFS-5203. Concurrent clients that add a cache directive on the same path
+    may prematurely uncache each other.  (Chris Nauroth via Colin Patrick McCabe)
+
+    HDFS-5378. In CacheReport, don't send genstamp and length on the wire
+    (Contributed by Colin Patrick McCabe)
+
+    HDFS-5385. Caching RPCs are AtMostOnce, but do not persist client ID and
+    call ID to edit log.  (Chris Nauroth via Colin Patrick McCabe)
+
+    HDFS-5404 Resolve regressions in Windows compatibility on HDFS-4949 branch.
+    Contributed by Chris Nauroth.
+
+    HDFS-5405. Fix possible RetryCache hang for caching RPC handlers in
+    FSNamesystem. (Contributed by Andrew Wang)
+
+    HDFS-5419. Fixup test-patch.sh warnings on HDFS-4949 branch. (wang)
+
+    HDFS-5386. Add feature documentation for datanode caching. Contributed by
+    Colin Patrick McCabe.
+
+    HDFS-5468. CacheAdmin help command does not recognize commands  (Stephen
+    Chu via Colin Patrick McCabe)
+
+    HDFS-5326. add modifyDirective to cacheAdmin (cmccabe)
+
+    HDFS-5394: Fix race conditions in DN caching and uncaching (cmccabe)
+
+    HDFS-5320. Add datanode caching metrics. Contributed by Andrew Wang.
+
+    HDFS-5482. DistributedFileSystem#listPathBasedCacheDirectives must support
+    relative paths. Contributed by Colin Patrick McCabe.
+
+    HDFS-5471. CacheAdmin -listPools fails when user lacks permissions to view
+    all pools (Andrew Wang via Colin Patrick McCabe)
+
+    HDFS-5450. better API for getting the cached blocks locations. Contributed
+    by Andrew Wang.
+
+    HDFS-5485. add command-line support for modifyDirective (cmccabe)
+
+    HDFS-5366. recaching improvements (cmccabe)
+
+    HDFS-5520. loading cache path directives from edit log doesnt update
+    nextEntryId (cmccabe)
+
+    HDFS-5512. CacheAdmin -listPools fails with NPE when user lacks permissions
+    to view all pools (awang via cmccabe)
+
+    HDFS-5513. CacheAdmin commands fail when using . as the path. Contributed
+    by Andrew Wang.
+
+    HDFS-5511. improve CacheManipulator interface to allow better unit testing
+    (cmccabe)
+
+    HDFS-5451. Add byte and file statistics to PathBasedCacheEntry. Contributed
+    by Colin Patrick McCabe.
+
+    HDFS-5473. Consistent naming of user-visible caching classes and methods
+    (cmccabe)
+
+    HDFS-5543. Fix narrow race condition in TestPathBasedCacheRequests
+    (cmccabe)
+
+    HDFS-5565. CacheAdmin help should match against non-dashed commands (wang
+    via cmccabe)
+
+    HDFS-5556. Add some more NameNode cache statistics, cache pool stats
+    (cmccabe)
+
+    HDFS-5562. TestCacheDirectives and TestFsDatasetCache should stub out
+    native mlock. Contributed by Colin Patrick McCabe and Akira Ajisaka.
+
+    HDFS-5430. Support TTL on CacheDirectives. Contributed by Andrew Wang.
+
+    HDFS-5555. CacheAdmin commands fail when first listed NameNode is in
+    Standby (jxiang via cmccabe)
+
+    HDFS-5626. dfsadmin report shows incorrect values (cmccabe)
+
+    HDFS-5630. Hook up cache directive and pool usage statistics. (wang)
+
+    HDFS-5665. Remove the unnecessary writeLock while initializing CacheManager
+    in FsNameSystem Ctor. (Uma Maheswara Rao G via Andrew Wang)
+
+    HDFS-5431. Support cachepool-based limit management in path-based caching.
+    (awang via cmccabe)
+
+    HDFS-5679. TestCacheDirectives should handle the case where native code is
+    not available. (wang)
+
+    HDFS-5636. Enforce a max TTL per cache pool (awang via cmccabe)
+
+    HDFS-5701. Fix the CacheAdmin -addPool -maxTtl option name. Contributed by
+    Stephen Chu.
+
+    HDFS-5708. The CacheManager throws a NPE in the DataNode logs when
+    processing cache reports that refer to a block not known to the BlockManager.
+    Contributed by Colin Patrick McCabe.
+
+    HDFS-5659. dfsadmin -report doesn't output cache information properly.
+    Contributed by Andrew Wang.
+
+    HDFS-5651. Remove dfs.namenode.caching.enabled and improve CRM locking.
+    Contributed by Colin Patrick McCabe.
+
+    HDFS-5589. Namenode loops caching and uncaching when data should be
+    uncached. (awang via cmccabe)
+
+    HDFS-5724. modifyCacheDirective logging audit log command wrongly as
+    addCacheDirective (Uma Maheswara Rao G via Colin Patrick McCabe)
+
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -1108,6 +1137,11 @@ Release 2.3.0 - UNRELEASED
     HDFS-5649. Unregister NFS and Mount service when NFS gateway is shutting down.
     (brandonli)
 
+    HDFS-5789. Some of snapshot APIs missing checkOperation double check in fsn. (umamahesh)
+
+    HDFS-5343. When cat command is issued on snapshot files getting unexpected result.
+    (Sathish via umamahesh)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES
@@ -1275,7 +1309,7 @@ Release 2.1.1-beta - 2013-09-23
     HDFS-5091. Support for spnego keytab separate from the JournalNode keytab 
     for secure HA. (jing9)
 
-    HDFS-5051. nn fails to download checkpointed image from snn in some
+    HDFS-5055. nn fails to download checkpointed image from snn in some
     setups. (Vinay and suresh via suresh)
 
     HDFS-4898. BlockPlacementPolicyWithNodeGroup.chooseRemoteRack() fails to

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml

@@ -361,5 +361,10 @@
       <Class name="org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor" />
       <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE" />
     </Match>
+    <Match>
+      <Class name="org.apache.hadoop.hdfs.DFSUtil"/>
+      <Method name="assertAllResultsEqual" />
+      <Bug pattern="NP_LOAD_OF_KNOWN_NULL_VALUE" />
+    </Match>
 
  </FindBugsFilter>

+ 33 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java

@@ -18,6 +18,8 @@
 package org.apache.hadoop.contrib.bkjournal;
 
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.JournalManager;
 import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
@@ -659,6 +661,37 @@ public class BookKeeperJournalManager implements JournalManager {
     }
   }
 
+  @Override
+  public void doPreUpgrade() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void doUpgrade(Storage storage) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public long getJournalCTime() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void doFinalize() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean canRollBack(StorageInfo storage, StorageInfo prevStorage,
+      int targetLayoutVersion) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void doRollback() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
   @Override
   public void close() throws IOException {
     try {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperAsHASharedDir.java

@@ -316,7 +316,7 @@ public class TestBookKeeperAsHASharedDir {
     } catch (IOException ioe) {
       LOG.info("Got expected exception", ioe);
       GenericTestUtils.assertExceptionContains(
-          "Cannot start an HA namenode with name dirs that need recovery", ioe);
+          "storage directory does not exist or is not accessible", ioe);
     }
   }
 

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

@@ -772,7 +772,7 @@ public class DFSClient implements java.io.Closeable {
         final long elapsed = Time.now() - getLastLeaseRenewal();
         if (elapsed > HdfsConstants.LEASE_HARDLIMIT_PERIOD) {
           LOG.warn("Failed to renew lease for " + clientName + " for "
-              + (elapsed/1000) + " seconds (>= soft-limit ="
+              + (elapsed/1000) + " seconds (>= hard-limit ="
               + (HdfsConstants.LEASE_HARDLIMIT_PERIOD/1000) + " seconds.) "
               + "Closing all files being written ...", e);
           closeAllFilesBeingWritten(true);

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

@@ -304,6 +304,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_NAMENODE_DEFAULT_AUDIT_LOGGER_NAME = "default";
   public static final String  DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_KEY = "dfs.namenode.audit.log.token.tracking.id";
   public static final boolean DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_DEFAULT = false;
+  public static final String  DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY = "dfs.namenode.audit.log.async";
+  public static final boolean DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT = false;
 
   // 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";

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

@@ -792,6 +792,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             currentNode = blockSeekTo(pos);
           }
           int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
+          if (locatedBlocks.isLastBlockComplete()) {
+            realLen = (int) Math.min(realLen, locatedBlocks.getFileLength());
+          }
           int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
           
           if (result >= 0) {

+ 44 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -43,6 +43,7 @@ import java.net.URISyntaxException;
 import java.security.SecureRandom;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
@@ -574,10 +575,24 @@ public class DFSUtil {
     }
     return ret;
   }
+  
+  /**
+   * Get all of the RPC addresses of the individual NNs in a given nameservice.
+   * 
+   * @param conf Configuration
+   * @param nsId the nameservice whose NNs addresses we want.
+   * @param defaultValue default address to return in case key is not found.
+   * @return A map from nnId -> RPC address of each NN in the nameservice.
+   */
+  public static Map<String, InetSocketAddress> getRpcAddressesForNameserviceId(
+      Configuration conf, String nsId, String defaultValue) {
+    return getAddressesForNameserviceId(conf, nsId, defaultValue,
+        DFS_NAMENODE_RPC_ADDRESS_KEY);
+  }
 
   private static Map<String, InetSocketAddress> getAddressesForNameserviceId(
       Configuration conf, String nsId, String defaultValue,
-      String[] keys) {
+      String... keys) {
     Collection<String> nnIds = getNameNodeIds(conf, nsId);
     Map<String, InetSocketAddress> ret = Maps.newHashMap();
     for (String nnId : emptyAsSingletonNull(nnIds)) {
@@ -1670,4 +1685,32 @@ public class DFSUtil {
     }
     return builder;
   }
+
+  /**
+   * Assert that all objects in the collection are equal. Returns silently if
+   * so, throws an AssertionError if any object is not equal. All null values
+   * are considered equal.
+   * 
+   * @param objects the collection of objects to check for equality.
+   */
+  public static void assertAllResultsEqual(Collection<?> objects) {
+    Object[] resultsArray = objects.toArray();
+    
+    if (resultsArray.length == 0)
+      return;
+    
+    for (int i = 0; i < resultsArray.length; i++) {
+      if (i == 0)
+        continue;
+      else {
+        Object currElement = resultsArray[i];
+        Object lastElement = resultsArray[i - 1];
+        if ((currElement == null && currElement != lastElement) ||
+            (currElement != null && !currElement.equals(lastElement))) {
+          throw new AssertionError("Not all elements match in results: " +
+            Arrays.toString(resultsArray));
+        }
+      }
+    }
+  }
 }

+ 58 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java

@@ -26,22 +26,29 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.StandbyException;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
@@ -305,4 +312,55 @@ public class HAUtil {
     DFSClient dfsClient = dfs.getClient();
     return RPC.getServerAddress(dfsClient.getNamenode());
   }
+  
+  /**
+   * Get an RPC proxy for each NN in an HA nameservice. Used when a given RPC
+   * call should be made on every NN in an HA nameservice, not just the active.
+   * 
+   * @param conf configuration
+   * @param nsId the nameservice to get all of the proxies for.
+   * @return a list of RPC proxies for each NN in the nameservice.
+   * @throws IOException in the event of error.
+   */
+  public static List<ClientProtocol> getProxiesForAllNameNodesInNameservice(
+      Configuration conf, String nsId) throws IOException {
+    Map<String, InetSocketAddress> nnAddresses =
+        DFSUtil.getRpcAddressesForNameserviceId(conf, nsId, null);
+    
+    List<ClientProtocol> namenodes = new ArrayList<ClientProtocol>();
+    for (InetSocketAddress nnAddress : nnAddresses.values()) {
+      NameNodeProxies.ProxyAndInfo<ClientProtocol> proxyInfo = null;
+      proxyInfo = NameNodeProxies.createNonHAProxy(conf,
+          nnAddress, ClientProtocol.class,
+          UserGroupInformation.getCurrentUser(), false);
+      namenodes.add(proxyInfo.getProxy());
+    }
+    return namenodes;
+  }
+  
+  /**
+   * Used to ensure that at least one of the given HA NNs is currently in the
+   * active state..
+   * 
+   * @param namenodes list of RPC proxies for each NN to check.
+   * @return true if at least one NN is active, false if all are in the standby state.
+   * @throws IOException in the event of error.
+   */
+  public static boolean isAtLeastOneActive(List<ClientProtocol> namenodes)
+      throws IOException {
+    for (ClientProtocol namenode : namenodes) {
+      try {
+        namenode.getFileInfo("/");
+        return true;
+      } catch (RemoteException re) {
+        IOException cause = re.unwrapRemoteException();
+        if (cause instanceof StandbyException) {
+          // This is expected to happen for a standby NN.
+        } else {
+          throw re;
+        }
+      }
+    }
+    return false;
+  }
 }

+ 14 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochR
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 
@@ -151,4 +152,17 @@ interface AsyncLogger {
    * StringBuilder. This is displayed on the NN web UI.
    */
   public void appendReport(StringBuilder sb);
+
+  public ListenableFuture<Void> doPreUpgrade();
+
+  public ListenableFuture<Void> doUpgrade(StorageInfo sInfo);
+
+  public ListenableFuture<Void> doFinalize();
+
+  public ListenableFuture<Boolean> canRollBack(StorageInfo storage,
+      StorageInfo prevStorage, int targetLayoutVersion);
+
+  public ListenableFuture<Void> doRollback();
+
+  public ListenableFuture<Long> getJournalCTime();
 }

+ 68 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJourna
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 
@@ -308,4 +309,71 @@ class AsyncLoggerSet {
     }
     return QuorumCall.create(calls);
   }
+  
+  QuorumCall<AsyncLogger, Void> doPreUpgrade() {
+    Map<AsyncLogger, ListenableFuture<Void>> calls =
+        Maps.newHashMap();
+    for (AsyncLogger logger : loggers) {
+      ListenableFuture<Void> future =
+          logger.doPreUpgrade();
+      calls.put(logger, future);
+    }
+    return QuorumCall.create(calls);
+  }
+
+  public QuorumCall<AsyncLogger, Void> doUpgrade(StorageInfo sInfo) {
+    Map<AsyncLogger, ListenableFuture<Void>> calls =
+        Maps.newHashMap();
+    for (AsyncLogger logger : loggers) {
+      ListenableFuture<Void> future =
+          logger.doUpgrade(sInfo);
+      calls.put(logger, future);
+    }
+    return QuorumCall.create(calls);
+  }
+
+  public QuorumCall<AsyncLogger, Void> doFinalize() {
+    Map<AsyncLogger, ListenableFuture<Void>> calls =
+        Maps.newHashMap();
+    for (AsyncLogger logger : loggers) {
+      ListenableFuture<Void> future =
+          logger.doFinalize();
+      calls.put(logger, future);
+    }
+    return QuorumCall.create(calls);
+  }
+
+  public QuorumCall<AsyncLogger, Boolean> canRollBack(StorageInfo storage,
+      StorageInfo prevStorage, int targetLayoutVersion) {
+    Map<AsyncLogger, ListenableFuture<Boolean>> calls =
+        Maps.newHashMap();
+    for (AsyncLogger logger : loggers) {
+      ListenableFuture<Boolean> future =
+          logger.canRollBack(storage, prevStorage, targetLayoutVersion);
+      calls.put(logger, future);
+    }
+    return QuorumCall.create(calls);
+  }
+
+  public QuorumCall<AsyncLogger, Void> doRollback() {
+    Map<AsyncLogger, ListenableFuture<Void>> calls =
+        Maps.newHashMap();
+    for (AsyncLogger logger : loggers) {
+      ListenableFuture<Void> future =
+          logger.doRollback();
+      calls.put(logger, future);
+    }
+    return QuorumCall.create(calls);
+  }
+
+  public QuorumCall<AsyncLogger, Long> getJournalCTime() {
+    Map<AsyncLogger, ListenableFuture<Long>> calls =
+        Maps.newHashMap();
+    for (AsyncLogger logger : loggers) {
+      ListenableFuture<Long> future = logger.getJournalCTime();
+      calls.put(logger, future);
+    }
+    return QuorumCall.create(calls);
+  }
+
 }

+ 68 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
 import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolPB;
 import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.qjournal.server.GetJournalEditServlet;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -564,6 +565,72 @@ public class IPCLoggerChannel implements AsyncLogger {
       }
     });
   }
+  
+  @Override
+  public ListenableFuture<Void> doPreUpgrade() {
+    return executor.submit(new Callable<Void>() {
+      @Override
+      public Void call() throws IOException {
+        getProxy().doPreUpgrade(journalId);
+        return null;
+      }
+    });
+  }
+  
+  @Override
+  public ListenableFuture<Void> doUpgrade(final StorageInfo sInfo) {
+    return executor.submit(new Callable<Void>() {
+      @Override
+      public Void call() throws IOException {
+        getProxy().doUpgrade(journalId, sInfo);
+        return null;
+      }
+    });
+  }
+  
+  @Override
+  public ListenableFuture<Void> doFinalize() {
+    return executor.submit(new Callable<Void>() {
+      @Override
+      public Void call() throws IOException {
+        getProxy().doFinalize(journalId);
+        return null;
+      }
+    });
+  }
+  
+  @Override
+  public ListenableFuture<Boolean> canRollBack(final StorageInfo storage,
+      final StorageInfo prevStorage, final int targetLayoutVersion) {
+    return executor.submit(new Callable<Boolean>() {
+      @Override
+      public Boolean call() throws IOException {
+        return getProxy().canRollBack(journalId, storage, prevStorage,
+            targetLayoutVersion);
+      }
+    });
+  }
+
+  @Override
+  public ListenableFuture<Void> doRollback() {
+    return executor.submit(new Callable<Void>() {
+      @Override
+      public Void call() throws IOException {
+        getProxy().doRollback(journalId);
+        return null;
+      }
+    });
+  }
+  
+  @Override
+  public ListenableFuture<Long> getJournalCTime() {
+    return executor.submit(new Callable<Long>() {
+      @Override
+      public Long call() throws IOException {
+        return getProxy().getJournalCTime(journalId);
+      }
+    });
+  }
 
   @Override
   public String toString() {
@@ -636,4 +703,5 @@ public class IPCLoggerChannel implements AsyncLogger {
   private boolean hasHttpServerEndPoint() {
    return httpServerURL != null;
   }
+
 }

+ 138 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java

@@ -34,10 +34,13 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
 import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
@@ -77,8 +80,14 @@ public class QuorumJournalManager implements JournalManager {
   // Since these don't occur during normal operation, we can
   // use rather lengthy timeouts, and don't need to make them
   // configurable.
-  private static final int FORMAT_TIMEOUT_MS = 60000;
-  private static final int HASDATA_TIMEOUT_MS = 60000;
+  private static final int FORMAT_TIMEOUT_MS            = 60000;
+  private static final int HASDATA_TIMEOUT_MS           = 60000;
+  private static final int CAN_ROLL_BACK_TIMEOUT_MS     = 60000;
+  private static final int FINALIZE_TIMEOUT_MS          = 60000;
+  private static final int PRE_UPGRADE_TIMEOUT_MS       = 60000;
+  private static final int ROLL_BACK_TIMEOUT_MS         = 60000;
+  private static final int UPGRADE_TIMEOUT_MS           = 60000;
+  private static final int GET_JOURNAL_CTIME_TIMEOUT_MS = 60000;
   
   private final Configuration conf;
   private final URI uri;
@@ -492,4 +501,131 @@ public class QuorumJournalManager implements JournalManager {
     return loggers;
   }
 
+  @Override
+  public void doPreUpgrade() throws IOException {
+    QuorumCall<AsyncLogger, Void> call = loggers.doPreUpgrade();
+    try {
+      call.waitFor(loggers.size(), loggers.size(), 0, PRE_UPGRADE_TIMEOUT_MS,
+          "doPreUpgrade");
+      
+      if (call.countExceptions() > 0) {
+        call.rethrowException("Could not do pre-upgrade of one or more JournalNodes");
+      }
+    } catch (InterruptedException e) {
+      throw new IOException("Interrupted waiting for doPreUpgrade() response");
+    } catch (TimeoutException e) {
+      throw new IOException("Timed out waiting for doPreUpgrade() response");
+    }
+  }
+
+  @Override
+  public void doUpgrade(Storage storage) throws IOException {
+    QuorumCall<AsyncLogger, Void> call = loggers.doUpgrade(storage);
+    try {
+      call.waitFor(loggers.size(), loggers.size(), 0, UPGRADE_TIMEOUT_MS,
+          "doUpgrade");
+      
+      if (call.countExceptions() > 0) {
+        call.rethrowException("Could not perform upgrade of one or more JournalNodes");
+      }
+    } catch (InterruptedException e) {
+      throw new IOException("Interrupted waiting for doUpgrade() response");
+    } catch (TimeoutException e) {
+      throw new IOException("Timed out waiting for doUpgrade() response");
+    }
+  }
+  
+  @Override
+  public void doFinalize() throws IOException {
+    QuorumCall<AsyncLogger, Void> call = loggers.doFinalize();
+    try {
+      call.waitFor(loggers.size(), loggers.size(), 0, FINALIZE_TIMEOUT_MS,
+          "doFinalize");
+      
+      if (call.countExceptions() > 0) {
+        call.rethrowException("Could not finalize one or more JournalNodes");
+      }
+    } catch (InterruptedException e) {
+      throw new IOException("Interrupted waiting for doFinalize() response");
+    } catch (TimeoutException e) {
+      throw new IOException("Timed out waiting for doFinalize() response");
+    }
+  }
+  
+  @Override
+  public boolean canRollBack(StorageInfo storage, StorageInfo prevStorage,
+      int targetLayoutVersion) throws IOException {
+    QuorumCall<AsyncLogger, Boolean> call = loggers.canRollBack(storage,
+        prevStorage, targetLayoutVersion);
+    try {
+      call.waitFor(loggers.size(), loggers.size(), 0, CAN_ROLL_BACK_TIMEOUT_MS,
+          "lockSharedStorage");
+      
+      if (call.countExceptions() > 0) {
+        call.rethrowException("Could not check if roll back possible for"
+            + " one or more JournalNodes");
+      }
+      
+      // Either they all return the same thing or this call fails, so we can
+      // just return the first result.
+      DFSUtil.assertAllResultsEqual(call.getResults().values());
+      for (Boolean result : call.getResults().values()) {
+        return result;
+      }
+    } catch (InterruptedException e) {
+      throw new IOException("Interrupted waiting for lockSharedStorage() " +
+          "response");
+    } catch (TimeoutException e) {
+      throw new IOException("Timed out waiting for lockSharedStorage() " +
+          "response");
+    }
+    
+    throw new AssertionError("Unreachable code.");
+  }
+
+  @Override
+  public void doRollback() throws IOException {
+    QuorumCall<AsyncLogger, Void> call = loggers.doRollback();
+    try {
+      call.waitFor(loggers.size(), loggers.size(), 0, ROLL_BACK_TIMEOUT_MS,
+          "doRollback");
+      
+      if (call.countExceptions() > 0) {
+        call.rethrowException("Could not perform rollback of one or more JournalNodes");
+      }
+    } catch (InterruptedException e) {
+      throw new IOException("Interrupted waiting for doFinalize() response");
+    } catch (TimeoutException e) {
+      throw new IOException("Timed out waiting for doFinalize() response");
+    }
+  }
+  
+  @Override
+  public long getJournalCTime() throws IOException {
+    QuorumCall<AsyncLogger, Long> call = loggers.getJournalCTime();
+    try {
+      call.waitFor(loggers.size(), loggers.size(), 0,
+          GET_JOURNAL_CTIME_TIMEOUT_MS, "getJournalCTime");
+      
+      if (call.countExceptions() > 0) {
+        call.rethrowException("Could not journal CTime for one "
+            + "more JournalNodes");
+      }
+      
+      // Either they all return the same thing or this call fails, so we can
+      // just return the first result.
+      DFSUtil.assertAllResultsEqual(call.getResults().values());
+      for (Long result : call.getResults().values()) {
+        return result;
+      }
+    } catch (InterruptedException e) {
+      throw new IOException("Interrupted waiting for getJournalCTime() " +
+          "response");
+    } catch (TimeoutException e) {
+      throw new IOException("Timed out waiting for getJournalCTime() " +
+          "response");
+    }
+    
+    throw new AssertionError("Unreachable code.");
+  }
 }

+ 14 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochR
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
 import org.apache.hadoop.hdfs.qjournal.server.JournalNode;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.JournalManager;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.security.KerberosInfo;
@@ -143,4 +144,17 @@ public interface QJournalProtocol {
    */
   public void acceptRecovery(RequestInfo reqInfo,
       SegmentStateProto stateToAccept, URL fromUrl) throws IOException;
+
+  public void doPreUpgrade(String journalId) throws IOException;
+
+  public void doUpgrade(String journalId, StorageInfo sInfo) throws IOException;
+
+  public void doFinalize(String journalId) throws IOException;
+
+  public Boolean canRollBack(String journalId, StorageInfo storage,
+      StorageInfo prevStorage, int targetLayoutVersion) throws IOException;
+
+  public void doRollback(String journalId) throws IOException;
+
+  public Long getJournalCTime(String journalId) throws IOException;
 }

+ 94 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java

@@ -17,17 +17,35 @@
  */
 package org.apache.hadoop.hdfs.qjournal.protocolPB;
 
+import java.io.IOException;
+import java.net.URL;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.AcceptRecoveryRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.AcceptRecoveryResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.CanRollBackRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.CanRollBackResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.DoFinalizeRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.DoFinalizeResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.DoPreUpgradeRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.DoPreUpgradeResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.DoRollbackRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.DoRollbackResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.DoUpgradeRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.DoUpgradeResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FinalizeLogSegmentRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FinalizeLogSegmentResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FormatRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FormatResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalCTimeRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalCTimeResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.HeartbeatRequestProto;
@@ -39,8 +57,6 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.JournalRe
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.JournalResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
-import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.AcceptRecoveryRequestProto;
-import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.AcceptRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PurgeLogsRequestProto;
@@ -48,13 +64,11 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PurgeLogs
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
+import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 
-import java.io.IOException;
-import java.net.URL;
-
 /**
  * Implementation for protobuf service that forwards requests
  * received on {@link JournalProtocolPB} to the 
@@ -244,4 +258,79 @@ public class QJournalProtocolServerSideTranslatorPB implements QJournalProtocolP
         reqInfo.hasCommittedTxId() ?
           reqInfo.getCommittedTxId() : HdfsConstants.INVALID_TXID);
   }
+
+
+  @Override
+  public DoPreUpgradeResponseProto doPreUpgrade(RpcController controller,
+      DoPreUpgradeRequestProto request) throws ServiceException {
+    try {
+      impl.doPreUpgrade(convert(request.getJid()));
+      return DoPreUpgradeResponseProto.getDefaultInstance();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public DoUpgradeResponseProto doUpgrade(RpcController controller,
+      DoUpgradeRequestProto request) throws ServiceException {
+    try {
+      impl.doUpgrade(convert(request.getJid()),
+          PBHelper.convert(request.getSInfo()));
+      return DoUpgradeResponseProto.getDefaultInstance();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public DoFinalizeResponseProto doFinalize(RpcController controller,
+      DoFinalizeRequestProto request) throws ServiceException {
+    try {
+      impl.doFinalize(convert(request.getJid()));
+      return DoFinalizeResponseProto.getDefaultInstance();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public CanRollBackResponseProto canRollBack(RpcController controller,
+      CanRollBackRequestProto request) throws ServiceException {
+    try {
+      Boolean result = impl.canRollBack(convert(request.getJid()),
+          PBHelper.convert(request.getStorage()),
+          PBHelper.convert(request.getPrevStorage()),
+          request.getTargetLayoutVersion());
+      return CanRollBackResponseProto.newBuilder()
+          .setCanRollBack(result)
+          .build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public DoRollbackResponseProto doRollback(RpcController controller, DoRollbackRequestProto request)
+      throws ServiceException {
+    try {
+      impl.doRollback(convert(request.getJid()));
+      return DoRollbackResponseProto.getDefaultInstance();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public GetJournalCTimeResponseProto getJournalCTime(RpcController controller,
+      GetJournalCTimeRequestProto request) throws ServiceException {
+    try {
+      Long resultCTime = impl.getJournalCTime(convert(request.getJid()));
+      return GetJournalCTimeResponseProto.newBuilder()
+          .setResultCTime(resultCTime)
+          .build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }

+ 92 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java

@@ -23,13 +23,23 @@ import java.net.URL;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.AcceptRecoveryRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.CanRollBackRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.CanRollBackResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.DoFinalizeRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.DoPreUpgradeRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.DoRollbackRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.DoUpgradeRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FinalizeLogSegmentRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FormatRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalCTimeRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalCTimeResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.HeartbeatRequestProto;
@@ -39,7 +49,6 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.JournalId
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.JournalRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
-import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.AcceptRecoveryRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PurgeLogsRequestProto;
@@ -47,6 +56,7 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.RequestIn
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.ipc.ProtobufHelper;
@@ -277,4 +287,85 @@ public class QJournalProtocolTranslatorPB implements ProtocolMetaInterface,
         RPC.getProtocolVersion(QJournalProtocolPB.class), methodName);
   }
 
+  @Override
+  public void doPreUpgrade(String jid) throws IOException {
+    try {
+      rpcProxy.doPreUpgrade(NULL_CONTROLLER,
+          DoPreUpgradeRequestProto.newBuilder()
+            .setJid(convertJournalId(jid))
+            .build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void doUpgrade(String journalId, StorageInfo sInfo) throws IOException {
+    try {
+      rpcProxy.doUpgrade(NULL_CONTROLLER,
+          DoUpgradeRequestProto.newBuilder()
+            .setJid(convertJournalId(journalId))
+            .setSInfo(PBHelper.convert(sInfo))
+            .build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+  @Override
+  public void doFinalize(String jid) throws IOException {
+    try {
+      rpcProxy.doFinalize(NULL_CONTROLLER,
+          DoFinalizeRequestProto.newBuilder()
+            .setJid(convertJournalId(jid))
+            .build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public Boolean canRollBack(String journalId, StorageInfo storage,
+      StorageInfo prevStorage, int targetLayoutVersion) throws IOException {
+    try {
+      CanRollBackResponseProto response = rpcProxy.canRollBack(
+          NULL_CONTROLLER,
+          CanRollBackRequestProto.newBuilder()
+            .setJid(convertJournalId(journalId))
+            .setStorage(PBHelper.convert(storage))
+            .setPrevStorage(PBHelper.convert(prevStorage))
+            .setTargetLayoutVersion(targetLayoutVersion)
+            .build());
+      return response.getCanRollBack();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void doRollback(String journalId) throws IOException {
+    try {
+      rpcProxy.doRollback(NULL_CONTROLLER,
+          DoRollbackRequestProto.newBuilder()
+            .setJid(convertJournalId(journalId))
+            .build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public Long getJournalCTime(String journalId) throws IOException {
+    try {
+      GetJournalCTimeResponseProto response = rpcProxy.getJournalCTime(
+          NULL_CONTROLLER,
+          GetJournalCTimeRequestProto.newBuilder()
+            .setJid(convertJournalId(journalId))
+            .build());
+      return response.getResultCTime();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
 }

+ 18 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/GetJournalEditServlet.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.FileJournalManager;
 import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
 import org.apache.hadoop.hdfs.server.namenode.GetImageServlet;
@@ -139,20 +140,26 @@ public class GetJournalEditServlet extends HttpServlet {
   private boolean checkStorageInfoOrSendError(JNStorage storage,
       HttpServletRequest request, HttpServletResponse response)
       throws IOException {
-    String myStorageInfoString = storage.toColonSeparatedString();
+    int myNsId = storage.getNamespaceID();
+    String myClusterId = storage.getClusterID();
+    
     String theirStorageInfoString = StringEscapeUtils.escapeHtml(
         request.getParameter(STORAGEINFO_PARAM));
 
-    if (theirStorageInfoString != null
-        && !myStorageInfoString.equals(theirStorageInfoString)) {
-      String msg = "This node has storage info '" + myStorageInfoString
-          + "' but the requesting node expected '"
-          + theirStorageInfoString + "'";
-      
-      response.sendError(HttpServletResponse.SC_FORBIDDEN, msg);
-      LOG.warn("Received an invalid request file transfer request from " +
-          request.getRemoteAddr() + ": " + msg);
-      return false;
+    if (theirStorageInfoString != null) {
+      int theirNsId = StorageInfo.getNsIdFromColonSeparatedString(
+          theirStorageInfoString);
+      String theirClusterId = StorageInfo.getClusterIdFromColonSeparatedString(
+          theirStorageInfoString);
+      if (myNsId != theirNsId || !myClusterId.equals(theirClusterId)) {
+        String msg = "This node has namespaceId '" + myNsId + " and clusterId '"
+            + myClusterId + "' but the requesting node expected '" + theirNsId
+            + "' and '" + theirClusterId + "'";
+        response.sendError(HttpServletResponse.SC_FORBIDDEN, msg);
+        LOG.warn("Received an invalid request file transfer request from " +
+            request.getRemoteAddr() + ": " + msg);
+        return false;
+      }
     }
     return true;
   }

+ 9 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java

@@ -130,6 +130,10 @@ class JNStorage extends Storage {
     return new File(sd.getCurrentDir(), "paxos");
   }
   
+  File getRoot() {
+    return sd.getRoot();
+  }
+  
   /**
    * Remove any log files and associated paxos files which are older than
    * the given txid.
@@ -182,12 +186,15 @@ class JNStorage extends Storage {
     unlockAll();
     sd.clearDirectory();
     writeProperties(sd);
+    createPaxosDir();
+    analyzeStorage();
+  }
+  
+  void createPaxosDir() throws IOException {
     if (!getPaxosDir().mkdirs()) {
       throw new IOException("Could not create paxos dir: " + getPaxosDir());
     }
-    analyzeStorage();
   }
-
   
   void analyzeStorage() throws IOException {
     this.state = sd.analyzeStorage(StartupOption.REGULAR, this);

+ 65 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java

@@ -37,12 +37,14 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.qjournal.protocol.JournalNotFormattedException;
 import org.apache.hadoop.hdfs.qjournal.protocol.JournalOutOfSyncException;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PersistedRecoveryPaxosData;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
 import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
 import org.apache.hadoop.hdfs.server.namenode.FileJournalManager;
 import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
@@ -73,7 +75,7 @@ import com.google.protobuf.TextFormat;
  * Each such journal is entirely independent despite being hosted by
  * the same JVM.
  */
-class Journal implements Closeable {
+public class Journal implements Closeable {
   static final Log LOG = LogFactory.getLog(Journal.class);
 
 
@@ -122,8 +124,8 @@ class Journal implements Closeable {
    */
   private BestEffortLongFile committedTxnId;
   
-  private static final String LAST_PROMISED_FILENAME = "last-promised-epoch";
-  private static final String LAST_WRITER_EPOCH = "last-writer-epoch";
+  public static final String LAST_PROMISED_FILENAME = "last-promised-epoch";
+  public static final String LAST_WRITER_EPOCH = "last-writer-epoch";
   private static final String COMMITTED_TXID_FILENAME = "committed-txid";
   
   private final FileJournalManager fjm;
@@ -627,7 +629,7 @@ class Journal implements Closeable {
   }
 
   /**
-   * @see QJournalProtocol#getEditLogManifest(String, long)
+   * @see QJournalProtocol#getEditLogManifest(String, long, boolean)
    */
   public RemoteEditLogManifest getEditLogManifest(long sinceTxId,
       boolean inProgressOk) throws IOException {
@@ -728,7 +730,7 @@ class Journal implements Closeable {
   }
   
   /**
-   * @see QJournalProtocol#acceptRecovery(RequestInfo, SegmentStateProto, URL)
+   * @see QJournalProtocol#acceptRecovery(RequestInfo, QJournalProtocolProtos.SegmentStateProto, URL)
    */
   public synchronized void acceptRecovery(RequestInfo reqInfo,
       SegmentStateProto segment, URL fromUrl)
@@ -980,4 +982,62 @@ class Journal implements Closeable {
       }
     }
   }
+
+  public synchronized void doPreUpgrade() throws IOException {
+    storage.getJournalManager().doPreUpgrade();
+  }
+
+  public synchronized void doUpgrade(StorageInfo sInfo) throws IOException {
+    long oldCTime = storage.getCTime();
+    storage.cTime = sInfo.cTime;
+    int oldLV = storage.getLayoutVersion();
+    storage.layoutVersion = sInfo.layoutVersion;
+    LOG.info("Starting upgrade of edits directory: "
+        + ".\n   old LV = " + oldLV
+        + "; old CTime = " + oldCTime
+        + ".\n   new LV = " + storage.getLayoutVersion()
+        + "; new CTime = " + storage.getCTime());
+    storage.getJournalManager().doUpgrade(storage);
+    storage.createPaxosDir();
+    
+    // Copy over the contents of the epoch data files to the new dir.
+    File currentDir = storage.getSingularStorageDir().getCurrentDir();
+    File previousDir = storage.getSingularStorageDir().getPreviousDir();
+    
+    PersistentLongFile prevLastPromisedEpoch = new PersistentLongFile(
+        new File(previousDir, LAST_PROMISED_FILENAME), 0);
+    PersistentLongFile prevLastWriterEpoch = new PersistentLongFile(
+        new File(previousDir, LAST_WRITER_EPOCH), 0);
+    
+    lastPromisedEpoch = new PersistentLongFile(
+        new File(currentDir, LAST_PROMISED_FILENAME), 0);
+    lastWriterEpoch = new PersistentLongFile(
+        new File(currentDir, LAST_WRITER_EPOCH), 0);
+    
+    lastPromisedEpoch.set(prevLastPromisedEpoch.get());
+    lastWriterEpoch.set(prevLastWriterEpoch.get());
+  }
+
+  public synchronized void doFinalize() throws IOException {
+    LOG.info("Finalizing upgrade for journal " 
+          + storage.getRoot() + "."
+          + (storage.getLayoutVersion()==0 ? "" :
+            "\n   cur LV = " + storage.getLayoutVersion()
+            + "; cur CTime = " + storage.getCTime()));
+    storage.getJournalManager().doFinalize();
+  }
+
+  public Boolean canRollBack(StorageInfo storage, StorageInfo prevStorage,
+      int targetLayoutVersion) throws IOException {
+    return this.storage.getJournalManager().canRollBack(storage, prevStorage,
+        targetLayoutVersion);
+  }
+
+  public void doRollback() throws IOException {
+    storage.getJournalManager().doRollback();
+  }
+
+  public Long getJournalCTime() throws IOException {
+    return storage.getJournalManager().getJournalCTime();
+  }
 }

+ 28 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager;
 import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
@@ -285,4 +286,31 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
     StringUtils.startupShutdownMessage(JournalNode.class, args, LOG);
     System.exit(ToolRunner.run(new JournalNode(), args));
   }
+
+  public void doPreUpgrade(String journalId) throws IOException {
+    getOrCreateJournal(journalId).doPreUpgrade();
+  }
+
+  public void doUpgrade(String journalId, StorageInfo sInfo) throws IOException {
+    getOrCreateJournal(journalId).doUpgrade(sInfo);
+  }
+
+  public void doFinalize(String journalId) throws IOException {
+    getOrCreateJournal(journalId).doFinalize();
+  }
+
+  public Boolean canRollBack(String journalId, StorageInfo storage,
+      StorageInfo prevStorage, int targetLayoutVersion) throws IOException {
+    return getOrCreateJournal(journalId).canRollBack(storage, prevStorage,
+        targetLayoutVersion);
+  }
+
+  public void doRollback(String journalId) throws IOException {
+    getOrCreateJournal(journalId).doRollback();
+  }
+
+  public Long getJournalCTime(String journalId) throws IOException {
+    return getOrCreateJournal(journalId).getJournalCTime();
+  }
+
 }

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

@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentSt
 import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
 import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolPB;
 import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -205,4 +206,35 @@ class JournalNodeRpcServer implements QJournalProtocol {
         .acceptRecovery(reqInfo, log, fromUrl);
   }
 
+  @Override
+  public void doPreUpgrade(String journalId) throws IOException {
+    jn.doPreUpgrade(journalId);
+  }
+
+  @Override
+  public void doUpgrade(String journalId, StorageInfo sInfo) throws IOException {
+    jn.doUpgrade(journalId, sInfo);
+  }
+
+  @Override
+  public void doFinalize(String journalId) throws IOException {
+    jn.doFinalize(journalId);
+  }
+
+  @Override
+  public Boolean canRollBack(String journalId, StorageInfo storage,
+      StorageInfo prevStorage, int targetLayoutVersion)
+      throws IOException {
+    return jn.canRollBack(journalId, storage, prevStorage, targetLayoutVersion);
+  }
+
+  @Override
+  public void doRollback(String journalId) throws IOException {
+    jn.doRollback(journalId);
+  }
+
+  @Override
+  public Long getJournalCTime(String journalId) throws IOException {
+    return jn.getJournalCTime(journalId);
+  }
 }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -337,6 +337,7 @@ public class Balancer {
         sock.connect(
             NetUtils.createSocketAddr(target.datanode.getXferAddr()),
             HdfsServerConstants.READ_TIMEOUT);
+        sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
         sock.setKeepAlive(true);
         
         OutputStream unbufOut = sock.getOutputStream();

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

@@ -79,7 +79,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    */
   protected int tolerateHeartbeatMultiplier;
 
-  BlockPlacementPolicyDefault(Configuration conf,  FSClusterStats stats,
+  protected BlockPlacementPolicyDefault(Configuration conf, FSClusterStats stats,
                            NetworkTopology clusterMap) {
     initialize(conf, stats, clusterMap);
   }

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

@@ -46,12 +46,12 @@ import org.apache.hadoop.net.NodeBase;
  */
 public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefault {
 
-  BlockPlacementPolicyWithNodeGroup(Configuration conf,  FSClusterStats stats,
+  protected BlockPlacementPolicyWithNodeGroup(Configuration conf,  FSClusterStats stats,
       NetworkTopology clusterMap) {
     initialize(conf, stats, clusterMap);
   }
 
-  BlockPlacementPolicyWithNodeGroup() {
+  protected BlockPlacementPolicyWithNodeGroup() {
   }
 
   @Override

+ 77 - 132
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.common;
 
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.RandomAccessFile;
@@ -26,26 +25,23 @@ import java.lang.management.ManagementFactory;
 import java.nio.channels.FileLock;
 import java.nio.channels.OverlappingFileLockException;
 import java.util.ArrayList;
-import java.util.List;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Properties;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.util.VersionInfo;
 
-import com.google.common.base.Preconditions;
-
 import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
 
 
 
@@ -82,7 +78,6 @@ public abstract class Storage extends StorageInfo {
   public static final int[] LAYOUT_VERSIONS_203 = {-19, -31};
 
   public    static final String STORAGE_FILE_LOCK     = "in_use.lock";
-  protected static final String STORAGE_FILE_VERSION  = "VERSION";
   public    static final String STORAGE_DIR_CURRENT   = "current";
   public    static final String STORAGE_DIR_PREVIOUS  = "previous";
   public    static final String STORAGE_TMP_REMOVED   = "removed.tmp";
@@ -126,22 +121,24 @@ public abstract class Storage extends StorageInfo {
   
   private class DirIterator implements Iterator<StorageDirectory> {
     StorageDirType dirType;
+    boolean includeShared;
     int prevIndex; // for remove()
     int nextIndex; // for next()
     
-    DirIterator(StorageDirType dirType) {
+    DirIterator(StorageDirType dirType, boolean includeShared) {
       this.dirType = dirType;
       this.nextIndex = 0;
       this.prevIndex = 0;
+      this.includeShared = includeShared;
     }
     
     @Override
     public boolean hasNext() {
       if (storageDirs.isEmpty() || nextIndex >= storageDirs.size())
         return false;
-      if (dirType != null) {
+      if (dirType != null || !includeShared) {
         while (nextIndex < storageDirs.size()) {
-          if (getStorageDir(nextIndex).getStorageDirType().isOfType(dirType))
+          if (shouldReturnNextDir())
             break;
           nextIndex++;
         }
@@ -156,9 +153,9 @@ public abstract class Storage extends StorageInfo {
       StorageDirectory sd = getStorageDir(nextIndex);
       prevIndex = nextIndex;
       nextIndex++;
-      if (dirType != null) {
+      if (dirType != null || !includeShared) {
         while (nextIndex < storageDirs.size()) {
-          if (getStorageDir(nextIndex).getStorageDirType().isOfType(dirType))
+          if (shouldReturnNextDir())
             break;
           nextIndex++;
         }
@@ -172,6 +169,12 @@ public abstract class Storage extends StorageInfo {
       storageDirs.remove(prevIndex); // remove last returned element
       hasNext(); // reset nextIndex to correct place
     }
+    
+    private boolean shouldReturnNextDir() {
+      StorageDirectory sd = getStorageDir(nextIndex);
+      return (dirType == null || sd.getStorageDirType().isOfType(dirType)) &&
+          (includeShared || !sd.isShared());
+    }
   }
   
   /**
@@ -203,7 +206,27 @@ public abstract class Storage extends StorageInfo {
    * them via the Iterator
    */
   public Iterator<StorageDirectory> dirIterator(StorageDirType dirType) {
-    return new DirIterator(dirType);
+    return dirIterator(dirType, true);
+  }
+  
+  /**
+   * Return all entries in storageDirs, potentially excluding shared dirs.
+   * @param includeShared whether or not to include shared dirs.
+   * @return an iterator over the configured storage dirs.
+   */
+  public Iterator<StorageDirectory> dirIterator(boolean includeShared) {
+    return dirIterator(null, includeShared);
+  }
+  
+  /**
+   * @param dirType all entries will be of this type of dir
+   * @param includeShared true to include any shared directories,
+   *        false otherwise
+   * @return an iterator over the configured storage dirs.
+   */
+  public Iterator<StorageDirectory> dirIterator(StorageDirType dirType,
+      boolean includeShared) {
+    return new DirIterator(dirType, includeShared);
   }
   
   public Iterable<StorageDirectory> dirIterable(final StorageDirType dirType) {
@@ -233,7 +256,9 @@ public abstract class Storage extends StorageInfo {
   @InterfaceAudience.Private
   public static class StorageDirectory implements FormatConfirmable {
     final File root;              // root directory
-    final boolean useLock;        // flag to enable storage lock
+    // whether or not this dir is shared between two separate NNs for HA, or
+    // between multiple block pools in the case of federation.
+    final boolean isShared;
     final StorageDirType dirType; // storage dir type
     FileLock lock;                // storage lock
 
@@ -241,11 +266,11 @@ public abstract class Storage extends StorageInfo {
     
     public StorageDirectory(File dir) {
       // default dirType is null
-      this(dir, null, true);
+      this(dir, null, false);
     }
     
     public StorageDirectory(File dir, StorageDirType dirType) {
-      this(dir, dirType, true);
+      this(dir, dirType, false);
     }
     
     public void setStorageUuid(String storageUuid) {
@@ -260,14 +285,14 @@ public abstract class Storage extends StorageInfo {
      * Constructor
      * @param dir directory corresponding to the storage
      * @param dirType storage directory type
-     * @param useLock true - enables locking on the storage directory and false
-     *          disables locking
+     * @param isShared whether or not this dir is shared between two NNs. true
+     *          disables locking on the storage directory, false enables locking
      */
-    public StorageDirectory(File dir, StorageDirType dirType, boolean useLock) {
+    public StorageDirectory(File dir, StorageDirType dirType, boolean isShared) {
       this.root = dir;
       this.lock = null;
       this.dirType = dirType;
-      this.useLock = useLock;
+      this.isShared = isShared;
     }
     
     /**
@@ -621,6 +646,10 @@ public abstract class Storage extends StorageInfo {
       
       return true;
     }
+    
+    public boolean isShared() {
+      return isShared;
+    }
 
 
     /**
@@ -635,7 +664,7 @@ public abstract class Storage extends StorageInfo {
      * @throws IOException if locking fails
      */
     public void lock() throws IOException {
-      if (!useLock) {
+      if (isShared()) {
         LOG.info("Locking is disabled");
         return;
       }
@@ -889,22 +918,6 @@ public abstract class Storage extends StorageInfo {
     public String toString();
   }
   
-  /**
-   * Get common storage fields.
-   * Should be overloaded if additional fields need to be get.
-   * 
-   * @param props
-   * @throws IOException
-   */
-  protected void setFieldsFromProperties(
-      Properties props, StorageDirectory sd) throws IOException {
-    setLayoutVersion(props, sd);
-    setNamespaceID(props, sd);
-    setStorageType(props, sd);
-    setcTime(props, sd);
-    setClusterId(props, layoutVersion, sd);
-  }
-  
   /**
    * Set common storage fields into the given properties object.
    * Should be overloaded if additional fields need to be set.
@@ -923,22 +936,29 @@ public abstract class Storage extends StorageInfo {
     }
     props.setProperty("cTime", String.valueOf(cTime));
   }
-
+  
   /**
-   * Read properties from the VERSION file in the given storage directory.
+   * Get common storage fields.
+   * Should be overloaded if additional fields need to be get.
+   * 
+   * @param props
+   * @throws IOException
    */
-  public void readProperties(StorageDirectory sd) throws IOException {
-    Properties props = readPropertiesFile(sd.getVersionFile());
-    setFieldsFromProperties(props, sd);
+  protected void setFieldsFromProperties(
+      Properties props, StorageDirectory sd) throws IOException {
+    super.setFieldsFromProperties(props, sd);
+    setStorageType(props, sd);
   }
-
-  /**
-   * Read properties from the the previous/VERSION file in the given storage directory.
-   */
-  public void readPreviousVersionProperties(StorageDirectory sd)
-      throws IOException {
-    Properties props = readPropertiesFile(sd.getPreviousVersionFile());
-    setFieldsFromProperties(props, sd);
+  
+  /** Validate and set storage type from {@link Properties}*/
+  protected void setStorageType(Properties props, StorageDirectory sd)
+      throws InconsistentFSStateException {
+    NodeType type = NodeType.valueOf(getProperty(props, sd, "storageType"));
+    if (!storageType.equals(type)) {
+      throw new InconsistentFSStateException(sd.root,
+          "node type is incompatible with others.");
+    }
+    storageType = type;
   }
 
   /**
@@ -947,10 +967,15 @@ public abstract class Storage extends StorageInfo {
   public void writeProperties(StorageDirectory sd) throws IOException {
     writeProperties(sd.getVersionFile(), sd);
   }
-
+  
   public void writeProperties(File to, StorageDirectory sd) throws IOException {
     Properties props = new Properties();
     setPropertiesFromFields(props, sd);
+    writeProperties(to, sd, props);
+  }
+
+  public static void writeProperties(File to, StorageDirectory sd,
+      Properties props) throws IOException {
     RandomAccessFile file = new RandomAccessFile(to, "rws");
     FileOutputStream out = null;
     try {
@@ -977,23 +1002,6 @@ public abstract class Storage extends StorageInfo {
       file.close();
     }
   }
-  
-  public static Properties readPropertiesFile(File from) throws IOException {
-    RandomAccessFile file = new RandomAccessFile(from, "rws");
-    FileInputStream in = null;
-    Properties props = new Properties();
-    try {
-      in = new FileInputStream(file.getFD());
-      file.seek(0);
-      props.load(in);
-    } finally {
-      if (in != null) {
-        in.close();
-      }
-      file.close();
-    }
-    return props;
-  }
 
   public static void rename(File from, File to) throws IOException {
     if (!from.renameTo(to))
@@ -1044,69 +1052,6 @@ public abstract class Storage extends StorageInfo {
       + "-" + Long.toString(storage.getCTime());
   }
   
-  String getProperty(Properties props, StorageDirectory sd,
-      String name) throws InconsistentFSStateException {
-    String property = props.getProperty(name);
-    if (property == null) {
-      throw new InconsistentFSStateException(sd.root, "file "
-          + STORAGE_FILE_VERSION + " has " + name + " missing.");
-    }
-    return property;
-  }
-  
-  /** Validate and set storage type from {@link Properties}*/
-  protected void setStorageType(Properties props, StorageDirectory sd)
-      throws InconsistentFSStateException {
-    NodeType type = NodeType.valueOf(getProperty(props, sd, "storageType"));
-    if (!storageType.equals(type)) {
-      throw new InconsistentFSStateException(sd.root,
-          "node type is incompatible with others.");
-    }
-    storageType = type;
-  }
-  
-  /** Validate and set ctime from {@link Properties}*/
-  protected void setcTime(Properties props, StorageDirectory sd)
-      throws InconsistentFSStateException {
-    cTime = Long.parseLong(getProperty(props, sd, "cTime"));
-  }
-
-  /** Validate and set clusterId from {@link Properties}*/
-  protected void setClusterId(Properties props, int layoutVersion,
-      StorageDirectory sd) throws InconsistentFSStateException {
-    // Set cluster ID in version that supports federation
-    if (LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
-      String cid = getProperty(props, sd, "clusterID");
-      if (!(clusterID.equals("") || cid.equals("") || clusterID.equals(cid))) {
-        throw new InconsistentFSStateException(sd.getRoot(),
-            "cluster Id is incompatible with others.");
-      }
-      clusterID = cid;
-    }
-  }
-  
-  /** Validate and set layout version from {@link Properties}*/
-  protected void setLayoutVersion(Properties props, StorageDirectory sd)
-      throws IncorrectVersionException, InconsistentFSStateException {
-    int lv = Integer.parseInt(getProperty(props, sd, "layoutVersion"));
-    if (lv < HdfsConstants.LAYOUT_VERSION) { // future version
-      throw new IncorrectVersionException(lv, "storage directory "
-          + sd.root.getAbsolutePath());
-    }
-    layoutVersion = lv;
-  }
-  
-  /** Validate and set namespaceID version from {@link Properties}*/
-  protected void setNamespaceID(Properties props, StorageDirectory sd)
-      throws InconsistentFSStateException {
-    int nsId = Integer.parseInt(getProperty(props, sd, "namespaceID"));
-    if (namespaceID != 0 && nsId != 0 && namespaceID != nsId) {
-      throw new InconsistentFSStateException(sd.root,
-          "namespaceID is incompatible with others.");
-    }
-    namespaceID = nsId;
-  }
-  
   public static boolean is203LayoutVersion(int layoutVersion) {
     for (int lv203 : LAYOUT_VERSIONS_203) {
       if (lv203 == layoutVersion) {

+ 119 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java

@@ -17,9 +17,17 @@
  */
 package org.apache.hadoop.hdfs.server.common;
 
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.util.Properties;
+
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 
 import com.google.common.base.Joiner;
 
@@ -34,6 +42,8 @@ public class StorageInfo {
   public int   namespaceID;     // id of the file system
   public String clusterID;      // id of the cluster
   public long  cTime;           // creation time of the file system state
+  
+  protected static final String STORAGE_FILE_VERSION    = "VERSION";
  
   public StorageInfo () {
     this(0, 0, "", 0L);
@@ -96,4 +106,113 @@ public class StorageInfo {
     return Joiner.on(":").join(
         layoutVersion, namespaceID, cTime, clusterID);
   }
+  
+  public static int getNsIdFromColonSeparatedString(String in) {
+    return Integer.parseInt(in.split(":")[1]);
+  }
+  
+  public static String getClusterIdFromColonSeparatedString(String in) {
+    return in.split(":")[3];
+  }
+  
+  /**
+   * Read properties from the VERSION file in the given storage directory.
+   */
+  public void readProperties(StorageDirectory sd) throws IOException {
+    Properties props = readPropertiesFile(sd.getVersionFile());
+    setFieldsFromProperties(props, sd);
+  }
+  
+  /**
+   * Read properties from the the previous/VERSION file in the given storage directory.
+   */
+  public void readPreviousVersionProperties(StorageDirectory sd)
+      throws IOException {
+    Properties props = readPropertiesFile(sd.getPreviousVersionFile());
+    setFieldsFromProperties(props, sd);
+  }
+  
+  /**
+   * Get common storage fields.
+   * Should be overloaded if additional fields need to be get.
+   * 
+   * @param props
+   * @throws IOException
+   */
+  protected void setFieldsFromProperties(
+      Properties props, StorageDirectory sd) throws IOException {
+    setLayoutVersion(props, sd);
+    setNamespaceID(props, sd);
+    setcTime(props, sd);
+    setClusterId(props, layoutVersion, sd);
+  }
+  
+  /** Validate and set ctime from {@link Properties}*/
+  protected void setcTime(Properties props, StorageDirectory sd)
+      throws InconsistentFSStateException {
+    cTime = Long.parseLong(getProperty(props, sd, "cTime"));
+  }
+
+  /** Validate and set clusterId from {@link Properties}*/
+  protected void setClusterId(Properties props, int layoutVersion,
+      StorageDirectory sd) throws InconsistentFSStateException {
+    // Set cluster ID in version that supports federation
+    if (LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
+      String cid = getProperty(props, sd, "clusterID");
+      if (!(clusterID.equals("") || cid.equals("") || clusterID.equals(cid))) {
+        throw new InconsistentFSStateException(sd.getRoot(),
+            "cluster Id is incompatible with others.");
+      }
+      clusterID = cid;
+    }
+  }
+  
+  /** Validate and set layout version from {@link Properties}*/
+  protected void setLayoutVersion(Properties props, StorageDirectory sd)
+      throws IncorrectVersionException, InconsistentFSStateException {
+    int lv = Integer.parseInt(getProperty(props, sd, "layoutVersion"));
+    if (lv < HdfsConstants.LAYOUT_VERSION) { // future version
+      throw new IncorrectVersionException(lv, "storage directory "
+          + sd.root.getAbsolutePath());
+    }
+    layoutVersion = lv;
+  }
+  
+  /** Validate and set namespaceID version from {@link Properties}*/
+  protected void setNamespaceID(Properties props, StorageDirectory sd)
+      throws InconsistentFSStateException {
+    int nsId = Integer.parseInt(getProperty(props, sd, "namespaceID"));
+    if (namespaceID != 0 && nsId != 0 && namespaceID != nsId) {
+      throw new InconsistentFSStateException(sd.root,
+          "namespaceID is incompatible with others.");
+    }
+    namespaceID = nsId;
+  }
+  
+  static String getProperty(Properties props, StorageDirectory sd,
+      String name) throws InconsistentFSStateException {
+    String property = props.getProperty(name);
+    if (property == null) {
+      throw new InconsistentFSStateException(sd.root, "file "
+          + STORAGE_FILE_VERSION + " has " + name + " missing.");
+    }
+    return property;
+  }
+  
+  public static Properties readPropertiesFile(File from) throws IOException {
+    RandomAccessFile file = new RandomAccessFile(from, "rws");
+    FileInputStream in = null;
+    Properties props = new Properties();
+    try {
+      in = new FileInputStream(file.getFD());
+      file.seek(0);
+      props.load(in);
+    } finally {
+      if (in != null) {
+        in.close();
+      }
+      file.close();
+    }
+    return props;
+  }
 }

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

@@ -103,7 +103,7 @@ public class BlockPoolSliceStorage extends Storage {
         dataDirs.size());
     for (Iterator<File> it = dataDirs.iterator(); it.hasNext();) {
       File dataDir = it.next();
-      StorageDirectory sd = new StorageDirectory(dataDir, null, false);
+      StorageDirectory sd = new StorageDirectory(dataDir, null, true);
       StorageState curState;
       try {
         curState = sd.analyzeStorage(startOpt, this);

+ 19 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java

@@ -23,6 +23,7 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.RandomAccessFile;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DU;
@@ -191,7 +192,7 @@ class BlockPoolSlice {
             blockFile.length(), genStamp, volume, blockFile.getParentFile());
       } else {
         newReplica = new ReplicaWaitingToBeRecovered(blockId,
-            validateIntegrity(blockFile, genStamp), 
+            validateIntegrityAndSetLength(blockFile, genStamp), 
             genStamp, volume, blockFile.getParentFile());
       }
 
@@ -214,7 +215,7 @@ class BlockPoolSlice {
    * @param genStamp generation stamp of the block
    * @return the number of valid bytes
    */
-  private long validateIntegrity(File blockFile, long genStamp) {
+  private long validateIntegrityAndSetLength(File blockFile, long genStamp) {
     DataInputStream checksumIn = null;
     InputStream blockIn = null;
     try {
@@ -257,11 +258,25 @@ class BlockPoolSlice {
       IOUtils.readFully(blockIn, buf, 0, lastChunkSize);
 
       checksum.update(buf, 0, lastChunkSize);
+      long validFileLength;
       if (checksum.compare(buf, lastChunkSize)) { // last chunk matches crc
-        return lastChunkStartPos + lastChunkSize;
+        validFileLength = lastChunkStartPos + lastChunkSize;
       } else { // last chunck is corrupt
-        return lastChunkStartPos;
+        validFileLength = lastChunkStartPos;
       }
+
+      // truncate if extra bytes are present without CRC
+      if (blockFile.length() > validFileLength) {
+        RandomAccessFile blockRAF = new RandomAccessFile(blockFile, "rw");
+        try {
+          // truncate blockFile
+          blockRAF.setLength(validFileLength);
+        } finally {
+          blockRAF.close();
+        }
+      }
+
+      return validFileLength;
     } catch (IOException e) {
       FsDatasetImpl.LOG.warn(e);
       return 0;

+ 33 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.IOException;
 import java.util.Collection;
 
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.JournalInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
@@ -97,4 +99,35 @@ class BackupJournalManager implements JournalManager {
   public String toString() {
     return "BackupJournalManager";
   }
+  
+  @Override
+  public void doPreUpgrade() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void doUpgrade(Storage storage) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  @Override
+  public void doFinalize() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean canRollBack(StorageInfo storage, StorageInfo prevStorage,
+      int targetLayoutVersion) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void doRollback() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public long getJournalCTime() throws IOException {
+    throw new UnsupportedOperationException();
+  }
 }

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

@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalProtoc
 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.HdfsServerConstants.StartupOption;
 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;
@@ -367,7 +368,7 @@ public class BackupNode extends NameNode {
     } else {
       nsInfo.validateStorage(storage);
     }
-    bnImage.initEditLog();
+    bnImage.initEditLog(StartupOption.REGULAR);
     setRegistration();
     NamenodeRegistration nnReg = null;
     while(!isStopRequested()) {
@@ -423,7 +424,8 @@ public class BackupNode extends NameNode {
     return DFSUtil.getBackupNameServiceId(conf);
   }
 
-  protected HAState createHAState() {
+  @Override
+  protected HAState createHAState(StartupOption startOpt) {
     return new BackupState();
   }
 

+ 23 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -174,7 +174,6 @@ public class FSDirectory implements Closeable {
         DFSConfigKeys.DFS_LIST_LIMIT, DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT);
     this.lsLimit = configuredLimit>0 ?
         configuredLimit : DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT;
-
     this.contentCountLimit = conf.getInt(
         DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_KEY,
         DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_DEFAULT);
@@ -1490,6 +1489,11 @@ public class FSDirectory implements Closeable {
   /**
    * Get a partial listing of the indicated directory
    *
+   * We will stop when any of the following conditions is met:
+   * 1) this.lsLimit files have been added
+   * 2) needLocation is true AND enough files have been added such
+   * that at least this.lsLimit block locations are in the response
+   *
    * @param src the directory name
    * @param startAfter the name to start listing after
    * @param needLocation if block locations are returned
@@ -1521,14 +1525,30 @@ public class FSDirectory implements Closeable {
       int startChild = INodeDirectory.nextChild(contents, startAfter);
       int totalNumChildren = contents.size();
       int numOfListing = Math.min(totalNumChildren-startChild, this.lsLimit);
+      int locationBudget = this.lsLimit;
+      int listingCnt = 0;
       HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
-      for (int i=0; i<numOfListing; i++) {
+      for (int i=0; i<numOfListing && locationBudget>0; i++) {
         INode cur = contents.get(startChild+i);
         listing[i] = createFileStatus(cur.getLocalNameBytes(), cur,
             needLocation, snapshot);
+        listingCnt++;
+        if (needLocation) {
+            // Once we  hit lsLimit locations, stop.
+            // This helps to prevent excessively large response payloads.
+            // Approximate #locations with locatedBlockCount() * repl_factor
+            LocatedBlocks blks = 
+                ((HdfsLocatedFileStatus)listing[i]).getBlockLocations();
+            locationBudget -= (blks == null) ? 0 :
+               blks.locatedBlockCount() * listing[i].getReplication();
+        }
+      }
+      // truncate return array if necessary
+      if (listingCnt < numOfListing) {
+          listing = Arrays.copyOf(listing, listingCnt);
       }
       return new DirectoryListing(
-          listing, totalNumChildren-startChild-numOfListing);
+          listing, totalNumChildren-startChild-listingCnt);
     } finally {
       readUnlock();
     }

+ 58 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
+import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp;
@@ -253,10 +254,12 @@ public class FSEditLog implements LogsPurgeable {
       if (u.getScheme().equals(NNStorage.LOCAL_URI_SCHEME)) {
         StorageDirectory sd = storage.getStorageDirectory(u);
         if (sd != null) {
-          journalSet.add(new FileJournalManager(conf, sd, storage), required);
+          journalSet.add(new FileJournalManager(conf, sd, storage),
+              required, sharedEditsDirs.contains(u));
         }
       } else {
-        journalSet.add(createJournal(u), required);
+        journalSet.add(createJournal(u), required,
+            sharedEditsDirs.contains(u));
       }
     }
  
@@ -1346,6 +1349,58 @@ public class FSEditLog implements LogsPurgeable {
     }
   }
   
+  public long getSharedLogCTime() throws IOException {
+    for (JournalAndStream jas : journalSet.getAllJournalStreams()) {
+      if (jas.isShared()) {
+        return jas.getManager().getJournalCTime();
+      }
+    }
+    throw new IOException("No shared log found.");
+  }
+  
+  public synchronized void doPreUpgradeOfSharedLog() throws IOException {
+    for (JournalAndStream jas : journalSet.getAllJournalStreams()) {
+      if (jas.isShared()) {
+        jas.getManager().doPreUpgrade();
+      }
+    }
+  }
+  
+  public synchronized void doUpgradeOfSharedLog() throws IOException {
+    for (JournalAndStream jas : journalSet.getAllJournalStreams()) {
+      if (jas.isShared()) {
+        jas.getManager().doUpgrade(storage);
+      }
+    }
+  }
+  
+  public synchronized void doFinalizeOfSharedLog() throws IOException {
+    for (JournalAndStream jas : journalSet.getAllJournalStreams()) {
+      if (jas.isShared()) {
+        jas.getManager().doFinalize();
+      }
+    }
+  }
+  
+  public synchronized boolean canRollBackSharedLog(Storage prevStorage,
+      int targetLayoutVersion) throws IOException {
+    for (JournalAndStream jas : journalSet.getAllJournalStreams()) {
+      if (jas.isShared()) {
+        return jas.getManager().canRollBack(storage, prevStorage,
+            targetLayoutVersion);
+      }
+    }
+    throw new IOException("No shared log found.");
+  }
+  
+  public synchronized void doRollback() throws IOException {
+    for (JournalAndStream jas : journalSet.getAllJournalStreams()) {
+      if (jas.isShared()) {
+        jas.getManager().doRollback();
+      }
+    }
+  }
+  
   @Override
   public void selectInputStreams(Collection<EditLogInputStream> streams,
       long fromTxId, boolean inProgressOk) throws IOException {
@@ -1477,4 +1532,5 @@ public class FSEditLog implements LogsPurgeable {
                                          + uri, e);
     }
   }
+
 }

+ 88 - 114
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -178,7 +178,8 @@ public class FSImage implements Closeable {
    * @return true if the image needs to be saved or false otherwise
    */
   boolean recoverTransitionRead(StartupOption startOpt, FSNamesystem target,
-      MetaRecoveryContext recovery) throws IOException {
+      MetaRecoveryContext recovery)
+      throws IOException {
     assert startOpt != StartupOption.FORMAT : 
       "NameNode formatting should be performed before reading the image";
     
@@ -252,8 +253,8 @@ public class FSImage implements Closeable {
       doImportCheckpoint(target);
       return false; // import checkpoint saved image already
     case ROLLBACK:
-      doRollback();
-      break;
+      throw new AssertionError("Rollback is now a standalone command, " +
+          "NameNode should not be starting with this option.");
     case REGULAR:
     default:
       // just load the image
@@ -272,17 +273,15 @@ public class FSImage implements Closeable {
   private boolean recoverStorageDirs(StartupOption startOpt,
       Map<StorageDirectory, StorageState> dataDirStates) throws IOException {
     boolean isFormatted = false;
+    // This loop needs to be over all storage dirs, even shared dirs, to make
+    // sure that we properly examine their state, but we make sure we don't
+    // mutate the shared dir below in the actual loop.
     for (Iterator<StorageDirectory> it = 
                       storage.dirIterator(); it.hasNext();) {
       StorageDirectory sd = it.next();
       StorageState curState;
       try {
         curState = sd.analyzeStorage(startOpt, storage);
-        String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
-        if (curState != StorageState.NORMAL && HAUtil.isHAEnabled(conf, nameserviceId)) {
-          throw new IOException("Cannot start an HA namenode with name dirs " +
-              "that need recovery. Dir: " + sd + " state: " + curState);
-        }
         // sd is locked but not opened
         switch(curState) {
         case NON_EXISTENT:
@@ -294,7 +293,7 @@ public class FSImage implements Closeable {
         case NORMAL:
           break;
         default:  // recovery is possible
-          sd.doRecover(curState);      
+          sd.doRecover(curState);
         }
         if (curState != StorageState.NOT_FORMATTED 
             && startOpt != StartupOption.ROLLBACK) {
@@ -315,10 +314,10 @@ public class FSImage implements Closeable {
     return isFormatted;
   }
 
-  private void doUpgrade(FSNamesystem target) throws IOException {
+  void doUpgrade(FSNamesystem target) throws IOException {
     // Upgrade is allowed only if there are 
-    // no previous fs states in any of the directories
-    for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) {
+    // no previous fs states in any of the local directories
+    for (Iterator<StorageDirectory> it = storage.dirIterator(false); it.hasNext();) {
       StorageDirectory sd = it.next();
       if (sd.getPreviousDir().exists())
         throw new InconsistentFSStateException(sd.getRoot(),
@@ -327,9 +326,10 @@ public class FSImage implements Closeable {
     }
 
     // load the latest image
-    this.loadFSImage(target, null, null);
 
     // Do upgrade for each directory
+    this.loadFSImage(target, StartupOption.UPGRADE, null);
+    
     long oldCTime = storage.getCTime();
     storage.cTime = now();  // generate new cTime for the state
     int oldLV = storage.getLayoutVersion();
@@ -337,28 +337,17 @@ public class FSImage implements Closeable {
     
     List<StorageDirectory> errorSDs =
       Collections.synchronizedList(new ArrayList<StorageDirectory>());
-    for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) {
+    assert !editLog.isSegmentOpen() : "Edits log must not be open.";
+    LOG.info("Starting upgrade of local storage directories."
+        + "\n   old LV = " + oldLV
+        + "; old CTime = " + oldCTime
+        + ".\n   new LV = " + storage.getLayoutVersion()
+        + "; new CTime = " + storage.getCTime());
+    // Do upgrade for each directory
+    for (Iterator<StorageDirectory> it = storage.dirIterator(false); it.hasNext();) {
       StorageDirectory sd = it.next();
-      LOG.info("Starting upgrade of image directory " + sd.getRoot()
-               + ".\n   old LV = " + oldLV
-               + "; old CTime = " + oldCTime
-               + ".\n   new LV = " + storage.getLayoutVersion()
-               + "; new CTime = " + storage.getCTime());
       try {
-        File curDir = sd.getCurrentDir();
-        File prevDir = sd.getPreviousDir();
-        File tmpDir = sd.getPreviousTmp();
-        assert curDir.exists() : "Current directory must exist.";
-        assert !prevDir.exists() : "previous directory must not exist.";
-        assert !tmpDir.exists() : "previous.tmp directory must not exist.";
-        assert !editLog.isSegmentOpen() : "Edits log must not be open.";
-
-        // rename current to tmp
-        NNStorage.rename(curDir, tmpDir);
-        
-        if (!curDir.mkdir()) {
-          throw new IOException("Cannot create directory " + curDir);
-        }
+        NNUpgradeUtil.doPreUpgrade(sd);
       } catch (Exception e) {
         LOG.error("Failed to move aside pre-upgrade storage " +
             "in image directory " + sd.getRoot(), e);
@@ -366,41 +355,38 @@ public class FSImage implements Closeable {
         continue;
       }
     }
+    if (target.isHaEnabled()) {
+      editLog.doPreUpgradeOfSharedLog();
+    }
     storage.reportErrorsOnDirectories(errorSDs);
     errorSDs.clear();
 
     saveFSImageInAllDirs(target, editLog.getLastWrittenTxId());
 
-    for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) {
+    for (Iterator<StorageDirectory> it = storage.dirIterator(false); it.hasNext();) {
       StorageDirectory sd = it.next();
       try {
-        // Write the version file, since saveFsImage above only makes the
-        // fsimage_<txid>, and the directory is otherwise empty.
-        storage.writeProperties(sd);
-        
-        File prevDir = sd.getPreviousDir();
-        File tmpDir = sd.getPreviousTmp();
-        // rename tmp to previous
-        NNStorage.rename(tmpDir, prevDir);
+        NNUpgradeUtil.doUpgrade(sd, storage);
       } catch (IOException ioe) {
-        LOG.error("Unable to rename temp to previous for " + sd.getRoot(), ioe);
         errorSDs.add(sd);
         continue;
       }
-      LOG.info("Upgrade of " + sd.getRoot() + " is complete.");
+    }
+    if (target.isHaEnabled()) {
+      editLog.doUpgradeOfSharedLog();
     }
     storage.reportErrorsOnDirectories(errorSDs);
-
+    
     isUpgradeFinalized = false;
     if (!storage.getRemovedStorageDirs().isEmpty()) {
-      //during upgrade, it's a fatal error to fail any storage directory
+      // during upgrade, it's a fatal error to fail any storage directory
       throw new IOException("Upgrade failed in "
           + storage.getRemovedStorageDirs().size()
           + " storage directory(ies), previously logged.");
     }
   }
 
-  private void doRollback() throws IOException {
+  void doRollback(FSNamesystem fsns) throws IOException {
     // Rollback is allowed only if there is 
     // a previous fs states in at least one of the storage directories.
     // Directories that don't have previous state do not rollback
@@ -408,85 +394,46 @@ public class FSImage implements Closeable {
     FSImage prevState = new FSImage(conf);
     try {
       prevState.getStorage().layoutVersion = HdfsConstants.LAYOUT_VERSION;
-      for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) {
+      for (Iterator<StorageDirectory> it = storage.dirIterator(false); it.hasNext();) {
         StorageDirectory sd = it.next();
-        File prevDir = sd.getPreviousDir();
-        if (!prevDir.exists()) {  // use current directory then
-          LOG.info("Storage directory " + sd.getRoot()
-            + " does not contain previous fs state.");
-          // read and verify consistency with other directories
-          storage.readProperties(sd);
+        if (!NNUpgradeUtil.canRollBack(sd, storage, prevState.getStorage(),
+            HdfsConstants.LAYOUT_VERSION)) {
           continue;
         }
-
-        // read and verify consistency of the prev dir
-        prevState.getStorage().readPreviousVersionProperties(sd);
-
-        if (prevState.getLayoutVersion() != HdfsConstants.LAYOUT_VERSION) {
-          throw new IOException(
-            "Cannot rollback to storage version " +
-                prevState.getLayoutVersion() +
-                " using this version of the NameNode, which uses storage version " +
-                HdfsConstants.LAYOUT_VERSION + ". " +
-              "Please use the previous version of HDFS to perform the rollback.");
-        }
         canRollback = true;
       }
+      
+      if (fsns.isHaEnabled()) {
+        // If HA is enabled, check if the shared log can be rolled back as well.
+        editLog.initJournalsForWrite();
+        canRollback |= editLog.canRollBackSharedLog(prevState.getStorage(),
+            HdfsConstants.LAYOUT_VERSION);
+      }
+      
       if (!canRollback)
         throw new IOException("Cannot rollback. None of the storage "
             + "directories contain previous fs state.");
-
+  
       // Now that we know all directories are going to be consistent
       // Do rollback for each directory containing previous state
-      for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) {
+      for (Iterator<StorageDirectory> it = storage.dirIterator(false); it.hasNext();) {
         StorageDirectory sd = it.next();
-        File prevDir = sd.getPreviousDir();
-        if (!prevDir.exists())
-          continue;
-
         LOG.info("Rolling back storage directory " + sd.getRoot()
-          + ".\n   new LV = " + prevState.getStorage().getLayoutVersion()
-          + "; new CTime = " + prevState.getStorage().getCTime());
-        File tmpDir = sd.getRemovedTmp();
-        assert !tmpDir.exists() : "removed.tmp directory must not exist.";
-        // rename current to tmp
-        File curDir = sd.getCurrentDir();
-        assert curDir.exists() : "Current directory must exist.";
-        NNStorage.rename(curDir, tmpDir);
-        // rename previous to current
-        NNStorage.rename(prevDir, curDir);
-
-        // delete tmp dir
-        NNStorage.deleteDir(tmpDir);
-        LOG.info("Rollback of " + sd.getRoot()+ " is complete.");
+                 + ".\n   new LV = " + prevState.getStorage().getLayoutVersion()
+                 + "; new CTime = " + prevState.getStorage().getCTime());
+        NNUpgradeUtil.doRollBack(sd);
+      }
+      if (fsns.isHaEnabled()) {
+        // If HA is enabled, try to roll back the shared log as well.
+        editLog.doRollback();
       }
+      
       isUpgradeFinalized = true;
     } finally {
       prevState.close();
     }
   }
 
-  private void doFinalize(StorageDirectory sd) throws IOException {
-    File prevDir = sd.getPreviousDir();
-    if (!prevDir.exists()) { // already discarded
-      LOG.info("Directory " + prevDir + " does not exist.");
-      LOG.info("Finalize upgrade for " + sd.getRoot()+ " is not required.");
-      return;
-    }
-    LOG.info("Finalizing upgrade for storage directory " 
-             + sd.getRoot() + "."
-             + (storage.getLayoutVersion()==0 ? "" :
-                   "\n   cur LV = " + storage.getLayoutVersion()
-                   + "; cur CTime = " + storage.getCTime()));
-    assert sd.getCurrentDir().exists() : "Current directory must exist.";
-    final File tmpDir = sd.getFinalizedTmp();
-    // rename previous to tmp and remove
-    NNStorage.rename(prevDir, tmpDir);
-    NNStorage.deleteDir(tmpDir);
-    isUpgradeFinalized = true;
-    LOG.info("Finalize upgrade for " + sd.getRoot()+ " is complete.");
-  }
-
   /**
    * Load image from a checkpoint directory and save it into the current one.
    * @param target the NameSystem to import into
@@ -521,7 +468,7 @@ public class FSImage implements Closeable {
     // return back the real image
     realImage.getStorage().setStorageInfo(ckptImage.getStorage());
     realImage.getEditLog().setNextTxId(ckptImage.getEditLog().getLastWrittenTxId()+1);
-    realImage.initEditLog();
+    realImage.initEditLog(StartupOption.IMPORT);
 
     target.dir.fsImage = realImage;
     realImage.getStorage().setBlockPoolID(ckptImage.getBlockPoolID());
@@ -530,12 +477,23 @@ public class FSImage implements Closeable {
     saveNamespace(target);
     getStorage().writeAll();
   }
-
-  void finalizeUpgrade() throws IOException {
-    for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) {
+  
+  void finalizeUpgrade(boolean finalizeEditLog) throws IOException {
+    LOG.info("Finalizing upgrade for local dirs. " +
+        (storage.getLayoutVersion() == 0 ? "" : 
+          "\n   cur LV = " + storage.getLayoutVersion()
+          + "; cur CTime = " + storage.getCTime()));
+    for (Iterator<StorageDirectory> it = storage.dirIterator(false); it.hasNext();) {
       StorageDirectory sd = it.next();
-      doFinalize(sd);
+      NNUpgradeUtil.doFinalize(sd);
     }
+    if (finalizeEditLog) {
+      // We only do this in the case that HA is enabled and we're active. In any
+      // other case the NN will have done the upgrade of the edits directories
+      // already by virtue of the fact that they're local.
+      editLog.doFinalizeOfSharedLog();
+    }
+    isUpgradeFinalized = true;
   }
 
   boolean isUpgradeFinalized() {
@@ -601,7 +559,7 @@ public class FSImage implements Closeable {
 
     Iterable<EditLogInputStream> editStreams = null;
 
-    initEditLog();
+    initEditLog(startOpt);
 
     if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT, 
                                getLayoutVersion())) {
@@ -683,14 +641,30 @@ public class FSImage implements Closeable {
     }
   }
 
-  public void initEditLog() {
+  public void initEditLog(StartupOption startOpt) throws IOException {
     Preconditions.checkState(getNamespaceID() != 0,
         "Must know namespace ID before initting edit log");
     String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
     if (!HAUtil.isHAEnabled(conf, nameserviceId)) {
+      // If this NN is not HA
       editLog.initJournalsForWrite();
       editLog.recoverUnclosedStreams();
+    } else if (HAUtil.isHAEnabled(conf, nameserviceId) &&
+        startOpt == StartupOption.UPGRADE) {
+      // This NN is HA, but we're doing an upgrade so init the edit log for
+      // write.
+      editLog.initJournalsForWrite();
+      long sharedLogCTime = editLog.getSharedLogCTime();
+      if (this.storage.getCTime() < sharedLogCTime) {
+        throw new IOException("It looks like the shared log is already " +
+            "being upgraded but this NN has not been upgraded yet. You " +
+            "should restart this NameNode with the '" +
+            StartupOption.BOOTSTRAPSTANDBY.getName() + "' option to bring " +
+            "this NN in sync with the other.");
+      }
+      editLog.recoverUnclosedStreams();
     } else {
+      // This NN is HA and we're not doing an upgrade.
       editLog.initSharedJournalsForRead();
     }
   }

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

@@ -38,6 +38,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECI
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DEFAULT_AUDIT_LOGGER_NAME;
@@ -122,6 +124,7 @@ import javax.management.StandardMBean;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
@@ -252,6 +255,9 @@ import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.VersionInfo;
+import org.apache.log4j.Appender;
+import org.apache.log4j.AsyncAppender;
+import org.apache.log4j.Logger;
 import org.mortbay.util.ajax.JSON;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -550,6 +556,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return leaseManager;
   }
   
+  boolean isHaEnabled() {
+    return haEnabled;
+  }
+  
   /**
    * Check the supplied configuration for correctness.
    * @param conf Supplies the configuration to validate.
@@ -661,6 +671,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
   FSNamesystem(Configuration conf, FSImage fsImage, boolean ignoreRetryCache)
       throws IOException {
+    if (conf.getBoolean(DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY,
+                        DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT)) {
+      LOG.info("Enabling async auditlog");
+      enableAsyncAuditLog();
+    }
     boolean fair = conf.getBoolean("dfs.namenode.fslock.fair", true);
     LOG.info("fsLock is fair:" + fair);
     fsLock = new FSNamesystemLock(fair);
@@ -874,7 +889,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       }
       // This will start a new log segment and write to the seen_txid file, so
       // we shouldn't do it when coming up in standby state
-      if (!haEnabled) {
+      if (!haEnabled || (haEnabled && startOpt == StartupOption.UPGRADE)) {
         fsImage.openEditLogForWrite();
       }
       success = true;
@@ -1000,6 +1015,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
         dir.fsImage.editLog.openForWrite();
       }
+      
       if (haEnabled) {
         // Renew all of the leases before becoming active.
         // This is because, while we were in standby mode,
@@ -1026,6 +1042,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
   }
 
+  private boolean inActiveState() {
+    return haContext != null &&
+        haContext.getState().getServiceState() == HAServiceState.ACTIVE;
+  }
+  
   /**
    * Initialize replication queues.
    */
@@ -1040,9 +1061,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    *         middle of the {@link #startActiveServices()}
    */
   public boolean inTransitionToActive() {
-    return haEnabled && haContext != null
-        && haContext.getState().getServiceState() == HAServiceState.ACTIVE
-        && startingActiveService;
+    return haEnabled && inActiveState() && startingActiveService;
   }
 
   private boolean shouldUseDelegationTokens() {
@@ -4519,11 +4538,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     
   void finalizeUpgrade() throws IOException {
     checkSuperuserPrivilege();
-    checkOperation(OperationCategory.WRITE);
+    checkOperation(OperationCategory.UNCHECKED);
     writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      getFSImage().finalizeUpgrade();
+      checkOperation(OperationCategory.UNCHECKED);
+      getFSImage().finalizeUpgrade(this.isHaEnabled() && inActiveState());
     } finally {
       writeUnlock();
     }
@@ -6783,6 +6802,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   
   /** Allow snapshot on a directroy. */
   void allowSnapshot(String path) throws SafeModeException, IOException {
+    checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -6808,6 +6828,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   
   /** Disallow snapshot on a directory. */
   void disallowSnapshot(String path) throws SafeModeException, IOException {
+    checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -6931,6 +6952,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
       throws IOException {
     SnapshottableDirectoryStatus[] status = null;
+    checkOperation(OperationCategory.READ);
     final FSPermissionChecker checker = getPermissionChecker();
     readLock();
     try {
@@ -6964,6 +6986,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   SnapshotDiffReport getSnapshotDiffReport(String path,
       String fromSnapshot, String toSnapshot) throws IOException {
     SnapshotDiffInfo diffs = null;
+    checkOperation(OperationCategory.READ);
     final FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
@@ -7445,5 +7468,27 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       auditLog.info(message);
     }
   }
+
+  private static void enableAsyncAuditLog() {
+    if (!(auditLog instanceof Log4JLogger)) {
+      LOG.warn("Log4j is required to enable async auditlog");
+      return;
+    }
+    Logger logger = ((Log4JLogger)auditLog).getLogger();
+    @SuppressWarnings("unchecked")
+    List<Appender> appenders = Collections.list(logger.getAllAppenders());
+    // failsafe against trying to async it more than once
+    if (!appenders.isEmpty() && !(appenders.get(0) instanceof AsyncAppender)) {
+      AsyncAppender asyncAppender = new AsyncAppender();
+      // change logger to have an async appender containing all the
+      // previously configured appenders
+      for (Appender appender : appenders) {
+        logger.removeAppender(appender);
+        asyncAppender.addAppender(appender);
+      }
+      logger.addAppender(asyncAppender);        
+    }
+  }
+
 }
 

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

@@ -33,14 +33,15 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager.StoragePurger;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
-
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
@@ -489,4 +490,49 @@ public class FileJournalManager implements JournalManager {
                            isInProgress(), hasCorruptHeader);
     }
   }
+  
+  @Override
+  public void doPreUpgrade() throws IOException {
+    LOG.info("Starting upgrade of edits directory " + sd.getRoot());
+    try {
+     NNUpgradeUtil.doPreUpgrade(sd);
+    } catch (IOException ioe) {
+     LOG.error("Failed to move aside pre-upgrade storage " +
+         "in image directory " + sd.getRoot(), ioe);
+     throw ioe;
+    }
+  }
+  
+  /**
+   * This method assumes that the fields of the {@link Storage} object have
+   * already been updated to the appropriate new values for the upgrade.
+   */
+  @Override
+  public void doUpgrade(Storage storage) throws IOException {
+    NNUpgradeUtil.doUpgrade(sd, storage);
+  }
+  
+  @Override
+  public void doFinalize() throws IOException {
+    NNUpgradeUtil.doFinalize(sd);
+  }
+
+  @Override
+  public boolean canRollBack(StorageInfo storage, StorageInfo prevStorage,
+      int targetLayoutVersion) throws IOException {
+    return NNUpgradeUtil.canRollBack(sd, storage,
+        prevStorage, targetLayoutVersion);
+  }
+
+  @Override
+  public void doRollback() throws IOException {
+    NNUpgradeUtil.doRollBack(sd);
+  }
+
+  @Override
+  public long getJournalCTime() throws IOException {
+    StorageInfo sInfo = new StorageInfo();
+    sInfo.readProperties(sd);
+    return sInfo.getCTime();
+  }
 }

+ 51 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java

@@ -22,7 +22,9 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 
 /**
@@ -64,6 +66,54 @@ public interface JournalManager extends Closeable, FormatConfirmable,
    * Recover segments which have not been finalized.
    */
   void recoverUnfinalizedSegments() throws IOException;
+  
+  /**
+   * Perform any steps that must succeed across all JournalManagers involved in
+   * an upgrade before proceeding onto the actual upgrade stage. If a call to
+   * any JM's doPreUpgrade method fails, then doUpgrade will not be called for
+   * any JM.
+   */
+  void doPreUpgrade() throws IOException;
+  
+  /**
+   * Perform the actual upgrade of the JM. After this is completed, the NN can
+   * begin to use the new upgraded metadata. This metadata may later be either
+   * finalized or rolled back to the previous state.
+   * 
+   * @param storage info about the new upgraded versions.
+   */
+  void doUpgrade(Storage storage) throws IOException;
+  
+  /**
+   * Finalize the upgrade. JMs should purge any state that they had been keeping
+   * around during the upgrade process. After this is completed, rollback is no
+   * longer allowed.
+   */
+  void doFinalize() throws IOException;
+  
+  /**
+   * Return true if this JM can roll back to the previous storage state, false
+   * otherwise. The NN will refuse to run the rollback operation unless at least
+   * one JM or fsimage storage directory can roll back.
+   * 
+   * @param storage the storage info for the current state
+   * @param prevStorage the storage info for the previous (unupgraded) state
+   * @param targetLayoutVersion the layout version we intend to roll back to
+   * @return true if this JM can roll back, false otherwise.
+   */
+  boolean canRollBack(StorageInfo storage, StorageInfo prevStorage,
+      int targetLayoutVersion) throws IOException;
+  
+  /**
+   * Perform the rollback to the previous FS state. JMs which do not need to
+   * roll back their state should just return without error.
+   */
+  void doRollback() throws IOException;
+  
+  /**
+   * @return the CTime of the journal manager.
+   */
+  long getJournalCTime() throws IOException;
 
   /**
    * Close the journal manager, freeing any resources it may hold.
@@ -84,4 +134,5 @@ public interface JournalManager extends Closeable, FormatConfirmable,
       super(reason);
     }
   }
+
 }

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

@@ -33,6 +33,8 @@ import java.util.concurrent.CopyOnWriteArrayList;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
@@ -77,11 +79,14 @@ public class JournalSet implements JournalManager {
     private final JournalManager journal;
     private boolean disabled = false;
     private EditLogOutputStream stream;
-    private boolean required = false;
+    private final boolean required;
+    private final boolean shared;
     
-    public JournalAndStream(JournalManager manager, boolean required) {
+    public JournalAndStream(JournalManager manager, boolean required,
+        boolean shared) {
       this.journal = manager;
       this.required = required;
+      this.shared = shared;
     }
 
     public void startLogSegment(long txId) throws IOException {
@@ -163,6 +168,10 @@ public class JournalSet implements JournalManager {
     public boolean isRequired() {
       return required;
     }
+    
+    public boolean isShared() {
+      return shared;
+    }
   }
  
   // COW implementation is necessary since some users (eg the web ui) call
@@ -178,7 +187,7 @@ public class JournalSet implements JournalManager {
   
   @Override
   public void format(NamespaceInfo nsInfo) throws IOException {
-    // The iteration is done by FSEditLog itself
+    // The operation is done by FSEditLog itself
     throw new UnsupportedOperationException();
   }
 
@@ -537,9 +546,13 @@ public class JournalSet implements JournalManager {
     }
     return jList;
   }
-
+  
   void add(JournalManager j, boolean required) {
-    JournalAndStream jas = new JournalAndStream(j, required);
+    add(j, required, false);
+  }
+  
+  void add(JournalManager j, boolean required, boolean shared) {
+    JournalAndStream jas = new JournalAndStream(j, required, shared);
     journals.add(jas);
   }
   
@@ -655,4 +668,40 @@ public class JournalSet implements JournalManager {
     }
     return buf.toString();
   }
+
+  @Override
+  public void doPreUpgrade() throws IOException {
+    // This operation is handled by FSEditLog directly.
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void doUpgrade(Storage storage) throws IOException {
+    // This operation is handled by FSEditLog directly.
+    throw new UnsupportedOperationException();
+  }
+  
+  @Override
+  public void doFinalize() throws IOException {
+    // This operation is handled by FSEditLog directly.
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean canRollBack(StorageInfo storage, StorageInfo prevStorage, int targetLayoutVersion) throws IOException {
+    // This operation is handled by FSEditLog directly.
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void doRollback() throws IOException {
+    // This operation is handled by FSEditLog directly.
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public long getJournalCTime() throws IOException {
+    // This operation is handled by FSEditLog directly.
+    throw new UnsupportedOperationException();
+  }
 }

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

@@ -299,7 +299,7 @@ public class NNStorage extends Storage implements Closeable,
       if(dirName.getScheme().compareTo("file") == 0) {
         this.addStorageDir(new StorageDirectory(new File(dirName.getPath()),
             dirType,
-            !sharedEditsDirs.contains(dirName))); // Don't lock the dir if it's shared.
+            sharedEditsDirs.contains(dirName))); // Don't lock the dir if it's shared.
       }
     }
 
@@ -310,7 +310,7 @@ public class NNStorage extends Storage implements Closeable,
       // URI is of type file://
       if(dirName.getScheme().compareTo("file") == 0)
         this.addStorageDir(new StorageDirectory(new File(dirName.getPath()),
-                    NameNodeDirType.EDITS, !sharedEditsDirs.contains(dirName)));
+                    NameNodeDirType.EDITS, sharedEditsDirs.contains(dirName)));
     }
   }
 
@@ -976,7 +976,7 @@ public class NNStorage extends Storage implements Closeable,
     StringBuilder layoutVersions = new StringBuilder();
 
     // First determine what range of layout versions we're going to inspect
-    for (Iterator<StorageDirectory> it = dirIterator();
+    for (Iterator<StorageDirectory> it = dirIterator(false);
          it.hasNext();) {
       StorageDirectory sd = it.next();
       if (!sd.getVersionFile().exists()) {

+ 174 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNUpgradeUtil.java

@@ -0,0 +1,174 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
+
+abstract class NNUpgradeUtil {
+  
+  private static final Log LOG = LogFactory.getLog(NNUpgradeUtil.class);
+  
+  /**
+   * Return true if this storage dir can roll back to the previous storage
+   * state, false otherwise. The NN will refuse to run the rollback operation
+   * unless at least one JM or fsimage storage directory can roll back.
+   * 
+   * @param storage the storage info for the current state
+   * @param prevStorage the storage info for the previous (unupgraded) state
+   * @param targetLayoutVersion the layout version we intend to roll back to
+   * @return true if this JM can roll back, false otherwise.
+   * @throws IOException in the event of error
+   */
+  static boolean canRollBack(StorageDirectory sd, StorageInfo storage,
+      StorageInfo prevStorage, int targetLayoutVersion) throws IOException {
+    File prevDir = sd.getPreviousDir();
+    if (!prevDir.exists()) {  // use current directory then
+      LOG.info("Storage directory " + sd.getRoot()
+               + " does not contain previous fs state.");
+      // read and verify consistency with other directories
+      storage.readProperties(sd);
+      return false;
+    }
+
+    // read and verify consistency of the prev dir
+    prevStorage.readPreviousVersionProperties(sd);
+
+    if (prevStorage.getLayoutVersion() != targetLayoutVersion) {
+      throw new IOException(
+        "Cannot rollback to storage version " +
+        prevStorage.getLayoutVersion() +
+        " using this version of the NameNode, which uses storage version " +
+        targetLayoutVersion + ". " +
+        "Please use the previous version of HDFS to perform the rollback.");
+    }
+    
+    return true;
+  }
+
+  /**
+   * Finalize the upgrade. The previous dir, if any, will be renamed and
+   * removed. After this is completed, rollback is no longer allowed.
+   * 
+   * @param sd the storage directory to finalize
+   * @throws IOException in the event of error
+   */
+  static void doFinalize(StorageDirectory sd) throws IOException {
+    File prevDir = sd.getPreviousDir();
+    if (!prevDir.exists()) { // already discarded
+      LOG.info("Directory " + prevDir + " does not exist.");
+      LOG.info("Finalize upgrade for " + sd.getRoot()+ " is not required.");
+      return;
+    }
+    LOG.info("Finalizing upgrade of storage directory " + sd.getRoot());
+    assert sd.getCurrentDir().exists() : "Current directory must exist.";
+    final File tmpDir = sd.getFinalizedTmp();
+    // rename previous to tmp and remove
+    NNStorage.rename(prevDir, tmpDir);
+    NNStorage.deleteDir(tmpDir);
+    LOG.info("Finalize upgrade for " + sd.getRoot()+ " is complete.");
+  }
+  
+  /**
+   * Perform any steps that must succeed across all storage dirs/JournalManagers
+   * involved in an upgrade before proceeding onto the actual upgrade stage. If
+   * a call to any JM's or local storage dir's doPreUpgrade method fails, then
+   * doUpgrade will not be called for any JM. The existing current dir is
+   * renamed to previous.tmp, and then a new, empty current dir is created.
+   * 
+   * @param sd the storage directory to perform the pre-upgrade procedure.
+   * @throws IOException in the event of error
+   */
+  static void doPreUpgrade(StorageDirectory sd) throws IOException {
+    LOG.info("Starting upgrade of storage directory " + sd.getRoot());
+    File curDir = sd.getCurrentDir();
+    File prevDir = sd.getPreviousDir();
+    File tmpDir = sd.getPreviousTmp();
+    assert curDir.exists() : "Current directory must exist.";
+    assert !prevDir.exists() : "previous directory must not exist.";
+    assert !tmpDir.exists() : "previous.tmp directory must not exist.";
+
+    // rename current to tmp
+    NNStorage.rename(curDir, tmpDir);
+    
+    if (!curDir.mkdir()) {
+      throw new IOException("Cannot create directory " + curDir);
+    }
+  }
+  
+  /**
+   * Perform the upgrade of the storage dir to the given storage info. The new
+   * storage info is written into the current directory, and the previous.tmp
+   * directory is renamed to previous.
+   * 
+   * @param sd the storage directory to upgrade
+   * @param storage info about the new upgraded versions.
+   * @throws IOException in the event of error
+   */
+  static void doUpgrade(StorageDirectory sd, Storage storage) throws
+      IOException {
+    LOG.info("Performing upgrade of storage directory " + sd.getRoot());
+    try {
+      // Write the version file, since saveFsImage only makes the
+      // fsimage_<txid>, and the directory is otherwise empty.
+      storage.writeProperties(sd);
+      
+      File prevDir = sd.getPreviousDir();
+      File tmpDir = sd.getPreviousTmp();
+      // rename tmp to previous
+      NNStorage.rename(tmpDir, prevDir);
+    } catch (IOException ioe) {
+      LOG.error("Unable to rename temp to previous for " + sd.getRoot(), ioe);
+      throw ioe;
+    }
+  }
+
+  /**
+   * Perform rollback of the storage dir to the previous state. The existing
+   * current dir is removed, and the previous dir is renamed to current.
+   * 
+   * @param sd the storage directory to roll back.
+   * @throws IOException in the event of error
+   */
+  static void doRollBack(StorageDirectory sd)
+      throws IOException {
+    File prevDir = sd.getPreviousDir();
+    if (!prevDir.exists())
+      return;
+
+    File tmpDir = sd.getRemovedTmp();
+    assert !tmpDir.exists() : "removed.tmp directory must not exist.";
+    // rename current to tmp
+    File curDir = sd.getCurrentDir();
+    assert curDir.exists() : "Current directory must exist.";
+    NNStorage.rename(curDir, tmpDir);
+    // rename previous to current
+    NNStorage.rename(prevDir, curDir);
+
+    // delete tmp dir
+    NNStorage.deleteDir(tmpDir);
+    LOG.info("Rollback of " + sd.getRoot() + " is complete.");
+  }
+  
+}

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

@@ -648,7 +648,7 @@ public class NameNode implements NameNodeStatusMXBean {
     String nsId = getNameServiceId(conf);
     String namenodeId = HAUtil.getNameNodeId(conf, nsId);
     this.haEnabled = HAUtil.isHAEnabled(conf, nsId);
-    state = createHAState();
+    state = createHAState(getStartupOption(conf));
     this.allowStaleStandbyReads = HAUtil.shouldAllowStandbyReads(conf);
     this.haContext = createHAContext();
     try {
@@ -670,8 +670,12 @@ public class NameNode implements NameNodeStatusMXBean {
     }
   }
 
-  protected HAState createHAState() {
-    return !haEnabled ? ACTIVE_STATE : STANDBY_STATE;
+  protected HAState createHAState(StartupOption startOpt) {
+    if (!haEnabled || startOpt == StartupOption.UPGRADE) {
+      return ACTIVE_STATE;
+    } else {
+      return STANDBY_STATE;
+    }
   }
 
   protected HAContext createHAContext() {
@@ -1023,26 +1027,28 @@ public class NameNode implements NameNodeStatusMXBean {
       }
     }
   }
-
-  private static boolean finalize(Configuration conf,
-                               boolean isConfirmationNeeded
-                               ) throws IOException {
+  
+  @VisibleForTesting
+  public static boolean doRollback(Configuration conf,
+      boolean isConfirmationNeeded) throws IOException {
     String nsId = DFSUtil.getNamenodeNameServiceId(conf);
     String namenodeId = HAUtil.getNameNodeId(conf, nsId);
     initializeGenericKeys(conf, nsId, namenodeId);
 
     FSNamesystem nsys = new FSNamesystem(conf, new FSImage(conf));
     System.err.print(
-        "\"finalize\" will remove the previous state of the files system.\n"
-        + "Recent upgrade will become permanent.\n"
-        + "Rollback option will not be available anymore.\n");
+        "\"rollBack\" will remove the current state of the file system,\n"
+        + "returning you to the state prior to initiating your recent.\n"
+        + "upgrade. This action is permanent and cannot be undone. If you\n"
+        + "are performing a rollback in an HA environment, you should be\n"
+        + "certain that no NameNode process is running on any host.");
     if (isConfirmationNeeded) {
-      if (!confirmPrompt("Finalize filesystem state?")) {
-        System.err.println("Finalize aborted.");
+      if (!confirmPrompt("Roll back file system state?")) {
+        System.err.println("Rollback aborted.");
         return true;
       }
     }
-    nsys.dir.fsImage.finalizeUpgrade();
+    nsys.dir.fsImage.doRollback(nsys);
     return false;
   }
 
@@ -1210,14 +1216,6 @@ public class NameNode implements NameNodeStatusMXBean {
       return null;
     }
     setStartupOption(conf, startOpt);
-    
-    if (HAUtil.isHAEnabled(conf, DFSUtil.getNamenodeNameServiceId(conf)) &&
-        (startOpt == StartupOption.UPGRADE ||
-         startOpt == StartupOption.ROLLBACK ||
-         startOpt == StartupOption.FINALIZE)) {
-      throw new HadoopIllegalArgumentException("Invalid startup option. " +
-          "Cannot perform DFS upgrade with HA enabled.");
-    }
 
     switch (startOpt) {
       case FORMAT: {
@@ -1233,10 +1231,17 @@ public class NameNode implements NameNodeStatusMXBean {
         return null;
       }
       case FINALIZE: {
-        boolean aborted = finalize(conf, true);
-        terminate(aborted ? 1 : 0);
+        System.err.println("Use of the argument '" + StartupOption.FINALIZE +
+            "' is no longer supported. To finalize an upgrade, start the NN " +
+            " and then run `hdfs dfsadmin -finalizeUpgrade'");
+        terminate(1);
         return null; // avoid javac warning
       }
+      case ROLLBACK: {
+        boolean aborted = doRollback(conf, true);
+        terminate(aborted ? 1 : 0);
+        return null; // avoid warning
+      }
       case BOOTSTRAPSTANDBY: {
         String toolArgs[] = Arrays.copyOfRange(argv, 1, argv.length);
         int rc = BootstrapStandby.run(toolArgs, conf);

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

@@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -192,7 +193,7 @@ public class BootstrapStandby implements Tool, Configurable {
     FSImage image = new FSImage(conf);
     try {
       image.getStorage().setStorageInfo(storage);
-      image.initEditLog();
+      image.initEditLog(StartupOption.REGULAR);
       assert image.getEditLog().isOpenForRead() :
         "Expected edit log to be open for read";
 

+ 20 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.tools;
 import java.io.File;
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.net.URI;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
@@ -46,6 +47,7 @@ import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -830,7 +832,24 @@ public class DFSAdmin extends FsShell {
    */
   public int finalizeUpgrade() throws IOException {
     DistributedFileSystem dfs = getDFS();
-    dfs.finalizeUpgrade();
+    
+    Configuration dfsConf = dfs.getConf();
+    URI dfsUri = dfs.getUri();
+    boolean isHaEnabled = HAUtil.isLogicalUri(dfsConf, dfsUri);
+    if (isHaEnabled) {
+      // In the case of HA, run finalizeUpgrade for all NNs in this nameservice
+      String nsId = dfsUri.getHost();
+      List<ClientProtocol> namenodes =
+          HAUtil.getProxiesForAllNameNodesInNameservice(dfsConf, nsId);
+      if (!HAUtil.isAtLeastOneActive(namenodes)) {
+        throw new IOException("Cannot finalize with no NameNode active");
+      }
+      for (ClientProtocol haNn : namenodes) {
+        haNn.finalizeUpgrade();
+      }
+    } else {
+      dfs.finalizeUpgrade();
+    }
     
     return 0;
   }

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

@@ -133,6 +133,72 @@ message IsFormattedResponseProto {
   required bool isFormatted = 1;
 }
 
+/**
+ * getJournalCTime()
+ */
+message GetJournalCTimeRequestProto {
+  required JournalIdProto jid = 1;
+}
+
+message GetJournalCTimeResponseProto {
+  required int64 resultCTime = 1;
+}
+
+/**
+ * doPreUpgrade()
+ */
+message DoPreUpgradeRequestProto {
+  required JournalIdProto jid = 1;
+}
+
+message DoPreUpgradeResponseProto {
+}
+
+/**
+ * doUpgrade()
+ */
+message DoUpgradeRequestProto {
+  required JournalIdProto jid = 1;
+  required StorageInfoProto sInfo = 2;
+}
+
+message DoUpgradeResponseProto {
+}
+
+/**
+ * doFinalize()
+ */
+message DoFinalizeRequestProto {
+  required JournalIdProto jid = 1;
+}
+
+message DoFinalizeResponseProto {
+}
+
+/**
+ * canRollBack()
+ */
+message CanRollBackRequestProto {
+  required JournalIdProto jid = 1;
+  required StorageInfoProto storage = 2;
+  required StorageInfoProto prevStorage = 3;
+  required int32 targetLayoutVersion = 4;
+}
+
+message CanRollBackResponseProto {
+  required bool canRollBack = 1;
+}
+
+/**
+ * doRollback()
+ */
+message DoRollbackRequestProto {
+  required JournalIdProto jid = 1;
+}
+
+message DoRollbackResponseProto {
+}
+
 /**
  * getJournalState()
  */
@@ -236,6 +302,18 @@ message AcceptRecoveryResponseProto {
 service QJournalProtocolService {
   rpc isFormatted(IsFormattedRequestProto) returns (IsFormattedResponseProto);
 
+  rpc getJournalCTime(GetJournalCTimeRequestProto) returns (GetJournalCTimeResponseProto);
+  
+  rpc doPreUpgrade(DoPreUpgradeRequestProto) returns (DoPreUpgradeResponseProto);
+  
+  rpc doUpgrade(DoUpgradeRequestProto) returns (DoUpgradeResponseProto);
+
+  rpc doFinalize(DoFinalizeRequestProto) returns (DoFinalizeResponseProto);
+
+  rpc canRollBack(CanRollBackRequestProto) returns (CanRollBackResponseProto);
+
+  rpc doRollback(DoRollbackRequestProto) returns (DoRollbackResponseProto);
+
   rpc getJournalState(GetJournalStateRequestProto) returns (GetJournalStateResponseProto);
 
   rpc newEpoch(NewEpochRequestProto) returns (NewEpochResponseProto);

+ 200 - 217
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html

@@ -23,25 +23,53 @@
 <title>Namenode information</title>
 </head>
 <body>
+
+<header class="navbar navbar-inverse bs-docs-nav" role="banner">
+<div class="container">
+  <div class="navbar-header">
+    <a href="http://hadoop.apache.org/core" class="navbar-brand">Hadoop</a>
+  </div>
+
+  <ul class="nav navbar-nav" id="ui-tabs">
+    <li><a href="#tab-overview">Overview</a></li>
+    <li><a href="#tab-datanode">Datanodes</a></li>
+    <li><a href="#tab-snapshot">Snapshot</a></li>
+    <li><a href="#tab-startup-progress">Startup Progress</a></li>
+    <li class="dropdown">
+      <a href="#" class="dropdown-toggle" data-toggle="dropdown">Utilities <b class="caret"></b></a>
+      <ul class="dropdown-menu">
+        <li><a href="explorer.html">Browse the file system</a></li>
+        <li><a href="logs">Logs</a></li>
+      </ul>
+    </li>
+  </ul>
+</div>
+</header>
+
 <div class="container">
-<div class="alert alert-danger" id="alert-panel" style="display:none">
-<button type="button" class="close" onclick="$('#alert-panel').hide();">&times;</button>
-<div class="alert-body" id="alert-panel-body"></div>
+
+<div id="alert-panel">
+  <div class="alert alert-danger">
+    <button type="button" class="close" onclick="$('#alert-panel').hide();">&times;</button>
+    <div class="alert-body" id="alert-panel-body"></div>
+  </div>
 </div>
-<div id="panel"></div>
+
+<div class="tab-content">
+  <div class="tab-pane" id="tab-overview"></div>
+  <div class="tab-pane" id="tab-datanode"></div>
+  <div class="tab-pane" id="tab-snapshot"></div>
+  <div class="tab-pane" id="tab-startup-progress"></div>
 </div>
+
 <div class="row">
-<hr />
-<div class="col-xs-2"><p><a href="http://hadoop.apache.org/core">Hadoop</a>, 2013.</p></div>
-<div class="col-xs-1 pull-right"><a style="color: #ddd" href="dfshealth.jsp">Legacy UI</a></div>
+  <hr />
+  <div class="col-xs-2"><p><a href="http://hadoop.apache.org/core">Hadoop</a>, 2014.</p></div>
+  <div class="col-xs-1 pull-right"><a style="color: #ddd" href="dfshealth.jsp">Legacy UI</a></div>
+</div>
 </div>
 
 <script type="text/x-dust-template" id="tmpl-dfshealth">
-<div class="page-header">
-  {#nnstat}
-  <h1>NameNode '{HostAndPort}' ({State})</h1>
-  {/nnstat}
-</div>
 
 {#nn}
 {@if cond="{DistinctVersionCount} > 1"}
@@ -71,238 +99,193 @@
 {/if}
 {/nn}
 
-<div class="panel panel-success">
-  <div class="panel-heading">Overview</div>
-  <div class="panel-body">
-    {#nn}
-    <table class="table table-bordered">
-      <tr><th>Started:</th><td>{NNStarted}</td></tr>
-      <tr><th>Version:</th><td>{Version}</td></tr>
-      <tr><th>Compiled:</th><td>{CompileInfo}</td></tr>
-      <tr><th>Cluster ID:</th><td>{ClusterId}</td></tr>
-      <tr><th>Block Pool ID:</th><td>{BlockPoolId}</td></tr>
-    </table>
-    {/nn}
-  </div>
-</div>
-
-<p><a href="explorer.html">Browse the filesystem</a></p>
-<p><a href="/logs/">NameNode Logs</a></p>
-
-<hr/>
-
-<div class="panel panel-success">
-  <div class="panel-heading">Cluster Summary</div>
-  <div class="panel-body">
+<div class="page-header"><h1>Overview {#nnstat}<small>'{HostAndPort}' ({State})</small>{/nnstat}</h1></div>
+{#nn}
+<table class="table table-bordered table-striped">
+  <tr><th>Started:</th><td>{NNStarted}</td></tr>
+  <tr><th>Version:</th><td>{Version}</td></tr>
+  <tr><th>Compiled:</th><td>{CompileInfo}</td></tr>
+  <tr><th>Cluster ID:</th><td>{ClusterId}</td></tr>
+  <tr><th>Block Pool ID:</th><td>{BlockPoolId}</td></tr>
+</table>
+{/nn}
 
-    <p>
-      Security is {#nnstat}{#SecurityEnabled}on{:else}off{/SecurityEnabled}{/nnstat}.</p>
-    <p>{#nn}{#Safemode}{.}{:else}Safemode is off.{/Safemode}{/nn}</p>
+<div class="page-header"><h1>Summary</h1></div>
+<p>
+  Security is {#nnstat}{#SecurityEnabled}on{:else}off{/SecurityEnabled}{/nnstat}.</p>
+<p>{#nn}{#Safemode}{.}{:else}Safemode is off.{/Safemode}{/nn}</p>
 
-    <p>
-      {#fs}
-      {FilesTotal} files and directories, {BlocksTotal} blocks = {@math key="{FilesTotal}" method="add" operand="{BlocksTotal}"/} total filesystem object(s).
-      {#helper_fs_max_objects/}
-      {/fs}
-    </p>
-    {#mem.HeapMemoryUsage}
-    <p>Heap Memory used {used|fmt_bytes} of {committed|fmt_bytes} Heap Memory. Max Heap Memory is {max|fmt_bytes}. </p>
-    {/mem.HeapMemoryUsage}
+<p>
+  {#fs}
+  {FilesTotal} files and directories, {BlocksTotal} blocks = {@math key="{FilesTotal}" method="add" operand="{BlocksTotal}"/} total filesystem object(s).
+  {#helper_fs_max_objects/}
+  {/fs}
+</p>
+{#mem.HeapMemoryUsage}
+<p>Heap Memory used {used|fmt_bytes} of {committed|fmt_bytes} Heap Memory. Max Heap Memory is {max|fmt_bytes}. </p>
+{/mem.HeapMemoryUsage}
 
-    {#mem.NonHeapMemoryUsage}
-    <p>Non Heap Memory used {used|fmt_bytes} of {committed|fmt_bytes} Commited Non Heap Memory. Max Non Heap Memory is {max|fmt_bytes}. </p>
-    {/mem.NonHeapMemoryUsage}
+{#mem.NonHeapMemoryUsage}
+<p>Non Heap Memory used {used|fmt_bytes} of {committed|fmt_bytes} Commited Non Heap Memory. Max Non Heap Memory is {max|fmt_bytes}. </p>
+{/mem.NonHeapMemoryUsage}
 
-    {#nn}
-    <table class="table table-bordered table-striped">
-      <tr><th> Configured Capacity:</th><td>{Total|fmt_bytes}</td></tr>
-      <tr><th> DFS Used:</th><td>{Used|fmt_bytes}</td></tr>
-      <tr><th> Non DFS Used:</th><td>{NonDfsUsedSpace|fmt_bytes}</td></tr>
-      <tr><th> DFS Remaining:</th><td>{Free|fmt_bytes}</td></tr>
-      <tr><th> DFS Used%:</th><td>{PercentUsed|fmt_percentage}</td></tr>
-      <tr><th> DFS Remaining%:</th><td>{PercentRemaining|fmt_percentage}</td></tr>
-      <tr><th> Block Pool Used:</th><td>{BlockPoolUsedSpace|fmt_bytes}</td></tr>
-      <tr><th> Block Pool Used%:</th><td>{PercentBlockPoolUsed|fmt_percentage}</td></tr>
-      <tr><th> DataNodes usages% (Min/Median/Max/stdDev): </th>
+{#nn}
+<table class="table table-bordered table-striped">
+  <tr><th> Configured Capacity:</th><td>{Total|fmt_bytes}</td></tr>
+  <tr><th> DFS Used:</th><td>{Used|fmt_bytes}</td></tr>
+  <tr><th> Non DFS Used:</th><td>{NonDfsUsedSpace|fmt_bytes}</td></tr>
+  <tr><th> DFS Remaining:</th><td>{Free|fmt_bytes}</td></tr>
+  <tr><th> DFS Used%:</th><td>{PercentUsed|fmt_percentage}</td></tr>
+  <tr><th> DFS Remaining%:</th><td>{PercentRemaining|fmt_percentage}</td></tr>
+  <tr><th> Block Pool Used:</th><td>{BlockPoolUsedSpace|fmt_bytes}</td></tr>
+  <tr><th> Block Pool Used%:</th><td>{PercentBlockPoolUsed|fmt_percentage}</td></tr>
+  <tr><th> DataNodes usages% (Min/Median/Max/stdDev): </th>
 	<td>{#NodeUsage.nodeUsage}{min} / {median} / {max} / {stdDev}{/NodeUsage.nodeUsage}</td></tr>
-      {/nn}
+{/nn}
 
-      {#fs}
-      <tr><th><a href="#nodelist-operation">Live Nodes</a></th><td>{NumLiveDataNodes} (Decommissioned: {NumDecomLiveDataNodes})</td></tr>
-      <tr><th><a href="#nodelist-operation">Dead Nodes</a></th><td>{NumDeadDataNodes} (Decommissioned: {NumDecomDeadDataNodes})</td></tr>
-      <tr><th><a href="#nodelist-decom">Decommissioning Nodes</a></th><td>{NumDecommissioningDataNodes}</td></tr>
-      <tr><th title="Excludes missing blocks.">Number of Under-Replicated Blocks</th><td>{UnderReplicatedBlocks}</td></tr>
-      {/fs}
-    </table>
-  </div>
-</div>
+{#fs}
+  <tr><th><a href="#tab-datanode">Live Nodes</a></th><td>{NumLiveDataNodes} (Decommissioned: {NumDecomLiveDataNodes})</td></tr>
+  <tr><th><a href="#tab-datanode">Dead Nodes</a></th><td>{NumDeadDataNodes} (Decommissioned: {NumDecomDeadDataNodes})</td></tr>
+  <tr><th><a href="#tab-datanode">Decommissioning Nodes</a></th><td>{NumDecommissioningDataNodes}</td></tr>
+  <tr><th title="Excludes missing blocks.">Number of Under-Replicated Blocks</th><td>{UnderReplicatedBlocks}</td></tr>
+{/fs}
+</table>
 
-<hr/>
-<div class="panel panel-success">
-  <div class="panel-heading">NameNode Journal Status</div>
-  <div class="panel-body">
-    <p><b>Current transaction ID:</b> {nn.JournalTransactionInfo.LastAppliedOrWrittenTxId}</p>
-    <table class="table" title="NameNode Journals">
-      <thead>
+<div class="page-header"><h1>Namenode Journal Status</h1></div>
+<p><b>Current transaction ID:</b> {nn.JournalTransactionInfo.LastAppliedOrWrittenTxId}</p>
+<table class="table" title="NameNode Journals">
+  <thead>
 	<tr><th>Journal Manager</th><th>State</th></tr>
-      </thead>
-      <tbody>
+  </thead>
+  <tbody>
 	{#nn.NameJournalStatus}
 	<tr><td>{manager}</td><td>{stream}</td></tr>
 	{/nn.NameJournalStatus}
-      </tbody>
-    </table>
-  </div>
-</div>
+  </tbody>
+</table>
 
-<hr/>
-<div class="panel panel-success">
-  <div class="panel-heading">NameNode Storage</div>
-  <div class="panel-body">
-    <table class="table" title="NameNode Storage">
-      <thead><tr><td><b>Storage Directory</b></td><td><b>Type</b></td><td><b>State</b></td></tr></thead>
-      {#nn.NameDirStatuses}
-      {#active}{#helper_dir_status type="Active"/}{/active}
-      {#failed}{#helper_dir_status type="Failed"/}{/failed}
-      {/nn.NameDirStatuses}
-    </table>
-  </div>
-</div>
-<hr/>
+<div class="page-header"><h1>NameNode Storage</h1></div>
+<table class="table" title="NameNode Storage">
+  <thead><tr><td><b>Storage Directory</b></td><td><b>Type</b></td><td><b>State</b></td></tr></thead>
+  {#nn.NameDirStatuses}
+  {#active}{#helper_dir_status type="Active"/}{/active}
+  {#failed}{#helper_dir_status type="Failed"/}{/failed}
+  {/nn.NameDirStatuses}
+</table>
+</script>
 
-<div class="panel panel-success">
-  <div class="panel-heading">Snapshot Summary</div>
-  <div class="panel-body">
-    {#fs.SnapshotStats}
-    <table class="table" title="Snapshot Summary">
-      <thead><tr><td><b>Snapshottable directories</b></td>
-	  <td><b>Snapshotted directories</b></td></tr>
-      </thead>
-      <tbody>
-	<tr>
-	  <td>{SnapshottableDirectories}</td>
-	  <td>{Snapshots}</td>
-	</tr>
-      </tbody>
-    </table>
-    {/fs.SnapshotStats}
-  </div>
-</div>
-<hr/>
+<script type="text/x-dust-template" id="tmpl-snapshot">
+<div class="page-header"><h1>Snapshot Summary</h1></div>
+<p><b>Snapshottable directories</b>: {SnapshottableDirectories}</p>
+<p><b>Snapshotted directories</b>: {Snapshots}</p>
+</script>
+
+<script type="text/x-dust-template" id="tmpl-datanode">
+<div class="page-header"><h1>Datanode Information</h1></div>
+<div class="page-header"><h1><small>In operation</small></h1></div>
+<small>
+<table class="table">
+  <thead>
+    <tr>
+      <th>Node</th>
+      <th>Last contact</th>
+      <th>Admin State</th>
+      <th>Capacity</th>
+      <th>Used</th>
+      <th>Non DFS Used</th>
+      <th>Remaining</th>
+      <th>Blocks</th>
+      <th>Block pool used</th>
+      <th>Failed Volumes</th>
+      <th>Version</th>
+    </tr>
+  </thead>
+  {#LiveNodes}
+  <tr>
+    <td>{name} ({xferaddr})</td>
+    <td>{lastContact}</td>
+    <td>{adminState}</td>
+    <td>{capacity|fmt_bytes}</td>
+    <td>{used|fmt_bytes}</td>
+    <td>{nonDfsUsedSpace|fmt_bytes}</td>
+    <td>{remaining|fmt_bytes}</td>
+    <td>{numBlocks}</td>
+    <td>{blockPoolUsed|fmt_bytes} ({blockPoolUsedPercent|fmt_percentage})</td>
+    <td>{volfails}</td>
+    <td>{version}</td>
+  </tr>
+  {/LiveNodes}
+  {#DeadNodes}
+  <tr class="danger">
+    <td>{name} ({xferaddr})</td>
+    <td>{lastContact}</td>
+    <td>Dead{?decomissioned}, Decomissioned{/decomissioned}</td>
+    <td>-</td>
+    <td>-</td>
+    <td>-</td>
+    <td>-</td>
+    <td>-</td>
+    <td>-</td>
+    <td>-</td>
+    <td>-</td>
+  </tr>
+  {/DeadNodes}
+</table>
+</small>
+
+<div class="page-header"><h1><small>Decomissioning</small></h1></div>
+<small>
+<table class="table">
+  <thead>
+    <tr>
+      <th>Node</th>
+      <th>Last contact</th>
+      <th>Under replicated blocks</th>
+      <th>Blocks with no live replicas</th>
+      <th>Under Replicated Blocks <br/>In files under construction</th>
+    </tr>
+  </thead>
+  {#DecomNodes}
+  <tr>
+    <td>{name} ({xferaddr})</td>
+    <td>{lastContact}</td>
+    <td>{underReplicatedBlocks}</td>
+    <td>{decommissionOnlyReplicas}</td>
+    <td>{underReplicateInOpenFiles}</td>
+  </tr>
+  {/DecomNodes}
+</table>
+</small>
+</script>
 
-{#startup}
-<div class="panel panel-success">
-  <div class="panel-heading">Startup Progress</div>
-  <div class="panel-body">
-    <p>Elapsed Time: {elapsedTime|fmt_time}, Percent Complete: {percentComplete|fmt_percentage}</p>
-    <table class="table">
-      <thead>
-	<tr>
+<script type="text/x-dust-template" id="tmpl-startup-progress">
+<div class="page-header"><h1>Startup Progress</h1></div>
+<p>Elapsed Time: {elapsedTime|fmt_time}, Percent Complete: {percentComplete|fmt_percentage}</p>
+<table class="table">
+  <thead>
+	<tr class="active">
 	  <th>Phase</th>
-	  <th>Completion</th>
-	  <th>Elapsed Time</th>
+	  <th style="text-align:center">Completion</th>
+	  <th style="text-align:center">Elapsed Time</th>
 	</tr>
-      </thead>
-      <tbody>
+  </thead>
+  <tbody>
 	{#phases}
 	<tr class="phase">
 	  <td class="startupdesc">{desc} {file} {size|fmt_bytes}</td>
-	  <td>{percentComplete|fmt_percentage}</td>
-	  <td>{elapsedTime|fmt_time}</td>
+	  <td style="text-align:center">{percentComplete|fmt_percentage}</td>
+	  <td style="text-align:center">{elapsedTime|fmt_time}</td>
 	</tr>
 	{#steps root_file=file}
 	<tr class="step">
 	  <td class="startupdesc">{stepDesc} {stepFile} {stepSize|fmt_bytes} ({count}/{total})</td>
-	  <td>{percentComplete|fmt_percentage}</td>
+	  <td style="text-align:center">{percentComplete|fmt_percentage}</td>
 	  <td></td>
 	</tr>
 	{/steps}
 	{/phases}
-    </table>
-  </div>
-</div>
-{/startup}
-
-<hr/>
-<div class="panel panel-success">
-  <div class="panel-heading">Datanode Information</div>
-  <div class="panel-body">
-    <div class="panel panel-default" id="nodelist-operation">
-      <div class="panel-heading">Nodes in operation</div>
-      <div class="panel-body">
-        <table class="table">
-          <thead>
-            <tr>
-              <th>Node</th>
-              <th>Last contact</th>
-              <th>Admin State</th>
-              <th>Capacity</th>
-              <th>Used</th>
-              <th>Non DFS Used</th>
-              <th>Remaining</th>
-              <th>Blocks</th>
-              <th>Block pool used</th>
-              <th>Failed Volumes</th>
-            </tr>
-          </thead>
-          {#nn.LiveNodes}
-          <tr>
-            <td>{name} ({xferaddr})</td>
-            <td>{lastContact}</td>
-            <td>{adminState}</td>
-            <td>{capacity|fmt_bytes}</td>
-            <td>{used|fmt_bytes}</td>
-            <td>{nonDfsUsedSpace|fmt_bytes}</td>
-            <td>{remaining|fmt_bytes}</td>
-            <td>{numBlocks}</td>
-            <td>{blockPoolUsed|fmt_bytes} ({blockPoolUsedPercent|fmt_percentage})</td>
-            <td>{volfails}</td>
-          </tr>
-          {/nn.LiveNodes}
-          {#nn.DeadNodes}
-          <tr class="danger">
-            <td>{name} ({xferaddr})</td>
-            <td>{lastContact}</td>
-            <td>Dead{?decomissioned}, Decomissioned{/decomissioned}</td>
-            <td>-</td>
-            <td>-</td>
-            <td>-</td>
-            <td>-</td>
-            <td>-</td>
-            <td>-</td>
-            <td>-</td>
-          </tr>
-          {/nn.DeadNodes}
-        </table>
-      </div>
-    </div>
-    <div class="panel panel-default" id="nodelist-decom">
-      <div class="panel-heading">Nodes being decomissioned</div>
-      <div class="panel-body">
-        <table class="table">
-          <thead>
-            <tr>
-              <th>Node</th>
-              <th>Last contact</th>
-              <th>Under replicated blocks</th>
-              <th>Blocks with no live replicas</th>
-              <th>Under Replicated Blocks <br/>In files under construction</th>
-            </tr>
-          </thead>
-          {#nn.DecomNodes}
-          <tr>
-            <td>{name} ({xferaddr})</td>
-            <td>{lastContact}</td>
-            <td>{underReplicatedBlocks}</td>
-            <td>{decommissionOnlyReplicas}</td>
-	    <td>{underReplicateInOpenFiles}</td>
-	  </tr>
-	  {/nn.DecomNodes}
-	</table>
-      </div>
-    </div>
-  </div>
-</div>
+  </tbody>
+</table>
 </script>
 
 <script type="text/javascript" src="/static/jquery-1.10.2.min.js">

+ 113 - 55
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js

@@ -18,10 +18,20 @@
 (function () {
   "use strict";
 
-  var data = {};
+  dust.loadSource(dust.compile($('#tmpl-dfshealth').html(), 'dfshealth'));
+  dust.loadSource(dust.compile($('#tmpl-startup-progress').html(), 'startup-progress'));
+  dust.loadSource(dust.compile($('#tmpl-datanode').html(), 'datanode-info'));
+  dust.loadSource(dust.compile($('#tmpl-snapshot').html(), 'snapshot-info'));
 
-  function render() {
-    var helpers = {
+  function load_overview() {
+    var BEANS = [
+      {"name": "nn",      "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeInfo"},
+      {"name": "nnstat",  "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus"},
+      {"name": "fs",      "url": "/jmx?qry=Hadoop:service=NameNode,name=FSNamesystemState"},
+      {"name": "mem",     "url": "/jmx?qry=java.lang:type=Memory"},
+    ];
+
+    var HELPERS = {
       'helper_fs_max_objects': function (chunk, ctx, bodies, params) {
         var o = ctx.current();
         if (o.MaxObjects > 0) {
@@ -37,35 +47,53 @@
       }
     };
 
-    var base = dust.makeBase(helpers);
+    var data = {};
 
-    dust.loadSource(dust.compile($('#tmpl-dfshealth').html(), 'dfshealth'));
-    dust.render('dfshealth', base.push(data), function(err, out) {
-      $('#panel').html(out);
-    });
-  }
+    // Workarounds for the fact that JMXJsonServlet returns non-standard JSON strings
+    function data_workaround(d) {
+      d.nn.JournalTransactionInfo = JSON.parse(d.nn.JournalTransactionInfo);
+      d.nn.NameJournalStatus = JSON.parse(d.nn.NameJournalStatus);
+      d.nn.NameDirStatuses = JSON.parse(d.nn.NameDirStatuses);
+      d.nn.NodeUsage = JSON.parse(d.nn.NodeUsage);
+      d.nn.CorruptFiles = JSON.parse(d.nn.CorruptFiles);
+      return d;
+    }
 
-  var BEANS = [
-    {"name": "nn",      "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeInfo"},
-    {"name": "nnstat",  "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus"},
-    {"name": "fs",      "url": "/jmx?qry=Hadoop:service=NameNode,name=FSNamesystemState"},
-    {"name": "mem",     "url": "/jmx?qry=java.lang:type=Memory"},
-    {"name": "startup", "url": "/startupProgress"}
-  ];
-
-  // Workarounds for the fact that JMXJsonServlet returns non-standard JSON strings
-  function data_workaround(d) {
-    function node_map_to_array(nodes) {
-      var res = [];
-      for (var n in nodes) {
-        var p = nodes[n];
-        p.name = n;
-        res.push(p);
-      }
-      return res;
+    load_json(
+      BEANS,
+      function(d) {
+        for (var k in d) {
+          data[k] = d[k].beans[0];
+        }
+        data = data_workaround(data);
+        render();
+      },
+      function (url, jqxhr, text, err) {
+        show_err_msg('<p>Failed to retrieve data from ' + url + ', cause: ' + err + '</p>');
+      });
+
+    function render() {
+      var base = dust.makeBase(HELPERS);
+      dust.render('dfshealth', base.push(data), function(err, out) {
+        $('#tab-overview').html(out);
+        $('a[href="#tab-datanode"]').click(load_datanode_info);
+        $('#ui-tabs a[href="#tab-overview"]').tab('show');
+      });
     }
+  }
+  $('#ui-tabs a[href="#tab-overview"]').click(load_overview);
 
-    function startup_progress_workaround(r) {
+  function show_err_msg(msg) {
+    $('#alert-panel-body').html(msg);
+    $('#alert-panel').show();
+  }
+
+  function ajax_error_handler(url, jqxhr, text, err) {
+    show_err_msg('<p>Failed to retrieve data from ' + url + ', cause: ' + err + '</p>');
+  }
+
+  function load_startup_progress() {
+    function workaround(r) {
       function rename_property(o, s, d) {
         if (o[s] !== undefined) {
           o[d] = o[s];
@@ -86,36 +114,66 @@
       });
       return r;
     }
+    $.get('/startupProgress', function (resp) {
+      var data = workaround(resp);
+      dust.render('startup-progress', data, function(err, out) {
+        $('#tab-startup-progress').html(out);
+        $('#ui-tabs a[href="#tab-startup-progress"]').tab('show');
+      });
+    }).error(ajax_error_handler);
+  }
+
+  $('#ui-tabs a[href="#tab-startup-progress"]').click(load_startup_progress);
+
+  function load_datanode_info() {
+    function workaround(r) {
+      function node_map_to_array(nodes) {
+        var res = [];
+        for (var n in nodes) {
+          var p = nodes[n];
+          p.name = n;
+          res.push(p);
+        }
+        return res;
+      }
 
-    d.nn.JournalTransactionInfo = JSON.parse(d.nn.JournalTransactionInfo);
-    d.nn.NameJournalStatus = JSON.parse(d.nn.NameJournalStatus);
-    d.nn.NameDirStatuses = JSON.parse(d.nn.NameDirStatuses);
-    d.nn.NodeUsage = JSON.parse(d.nn.NodeUsage);
-    d.nn.LiveNodes = node_map_to_array(JSON.parse(d.nn.LiveNodes));
-    d.nn.DeadNodes = node_map_to_array(JSON.parse(d.nn.DeadNodes));
-    d.nn.DecomNodes = node_map_to_array(JSON.parse(d.nn.DecomNodes));
-    d.nn.CorruptFiles = JSON.parse(d.nn.CorruptFiles);
-
-    d.fs.SnapshotStats = JSON.parse(d.fs.SnapshotStats);
-    d.startup = startup_progress_workaround(d.startup);
-    return d;
+      r.LiveNodes = node_map_to_array(JSON.parse(r.LiveNodes));
+      r.DeadNodes = node_map_to_array(JSON.parse(r.DeadNodes));
+      r.DecomNodes = node_map_to_array(JSON.parse(r.DecomNodes));
+      return r;
+    }
+
+    $.get('/jmx?qry=Hadoop:service=NameNode,name=NameNodeInfo', function (resp) {
+      var data = workaround(resp.beans[0]);
+      dust.render('datanode-info', data, function(err, out) {
+        $('#tab-datanode').html(out);
+        $('#ui-tabs a[href="#tab-datanode"]').tab('show');
+      });
+    }).error(ajax_error_handler);
   }
 
-  function show_err_msg(msg) {
-    $('#alert-panel-body').html(msg);
-    $('#alert-panel').show();
+  $('a[href="#tab-datanode"]').click(load_datanode_info);
+
+  function load_snapshot_info() {
+    $.get('/jmx?qry=Hadoop:service=NameNode,name=FSNamesystemState', function (resp) {
+      var data = JSON.parse(resp.beans[0].SnapshotStats);
+      dust.render('snapshot-info', data, function(err, out) {
+        $('#tab-snapshot').html(out);
+        $('#ui-tabs a[href="#tab-snapshot"]').tab('show');
+      });
+    }).error(ajax_error_handler);
   }
 
-  load_json(
-    BEANS,
-    function(d) {
-      for (var k in d) {
-        data[k] = k === "startup" ? d[k] : d[k].beans[0];
-      }
-      data = data_workaround(data);
-      render();
-    },
-    function (url, jqxhr, text, err) {
-      show_err_msg('<p>Failed to retrieve data from ' + url + ', cause: ' + err + '</p>');
-    });
+  $('#ui-tabs a[href="#tab-snapshot"]').click(load_snapshot_info);
+
+  var hash = window.location.hash;
+  if (hash === "#tab-datanode") {
+    load_datanode_info();
+  } else if (hash === "#tab-snapshot") {
+    load_snapshot_info();
+  } else if (hash === "#tab-startup-progress") {
+    load_startup_progress();
+  } else {
+    load_overview();
+  }
 })();

+ 20 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/hadoop.css

@@ -192,4 +192,24 @@ div.security {
 .panel-success > .panel-heading {
   color: #fff !important;
   background-color: #5FA33E !important;
+}
+
+header.bs-docs-nav, header.bs-docs-nav .navbar-brand {
+  border-radius: 0px;
+  background-color: #5fa33e;
+  color: #fff;
+}
+
+#ui-tabs > li > a {
+  color: #dcf0d3;
+}
+
+#ui-tabs .active a {
+  color: #fff;
+  background-color: #446633;
+}
+
+#alert-panel {
+  margin-top:20px;
+  display: none;
 }

+ 46 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSHighAvailabilityWithQJM.apt.vm

@@ -765,3 +765,49 @@ digest:hdfs-zkfcs:vlUvLnd8MlacsE80rDuu6ONESbM=:rwcda
   Even if automatic failover is configured, you may initiate a manual failover
   using the same <<<hdfs haadmin>>> command. It will perform a coordinated
   failover.
+
+* HDFS Upgrade/Finalization/Rollback with HA Enabled
+
+  When moving between versions of HDFS, sometimes the newer software can simply
+  be installed and the cluster restarted. Sometimes, however, upgrading the
+  version of HDFS you're running may require changing on-disk data. In this case,
+  one must use the HDFS Upgrade/Finalize/Rollback facility after installing the
+  new software. This process is made more complex in an HA environment, since the
+  on-disk metadata that the NN relies upon is by definition distributed, both on
+  the two HA NNs in the pair, and on the JournalNodes in the case that QJM is
+  being used for the shared edits storage. This documentation section describes
+  the procedure to use the HDFS Upgrade/Finalize/Rollback facility in an HA setup.
+
+  <<To perform an HA upgrade>>, the operator must do the following:
+
+    [[1]] Shut down all of the NNs as normal, and install the newer software.
+
+    [[2]] Start one of the NNs with the <<<'-upgrade'>>> flag.
+  
+    [[3]] On start, this NN will not enter the standby state as usual in an HA
+    setup. Rather, this NN will immediately enter the active state, perform an
+    upgrade of its local storage dirs, and also perform an upgrade of the shared
+    edit log.
+  
+    [[4]] At this point the other NN in the HA pair will be out of sync with
+    the upgraded NN. In order to bring it back in sync and once again have a highly
+    available setup, you should re-bootstrap this NameNode by running the NN with
+    the <<<'-bootstrapStandby'>>> flag. It is an error to start this second NN with
+    the <<<'-upgrade'>>> flag.
+  
+  Note that if at any time you want to restart the NameNodes before finalizing
+  or rolling back the upgrade, you should start the NNs as normal, i.e. without
+  any special startup flag.
+
+  <<To finalize an HA upgrade>>, the operator will use the <<<`hdfsadmin
+  dfsadmin -finalizeUpgrade'>>> command while the NNs are running and one of them
+  is active. The active NN at the time this happens will perform the finalization
+  of the shared log, and the NN whose local storage directories contain the
+  previous FS state will delete its local state.
+
+  <<To perform a rollback>> of an upgrade, both NNs should first be shut down.
+  The operator should run the roll back command on the NN where they initiated
+  the upgrade procedure, which will perform the rollback on the local dirs there,
+  as well as on the shared log, either NFS or on the JNs. Afterward, this NN
+  should be started and the operator should run <<<`-bootstrapStandby'>>> on the
+  other NN to bring the two NNs in sync with this rolled-back file system state.

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

@@ -146,6 +146,7 @@ public class MiniDFSCluster {
     private boolean enableManagedDfsDirsRedundancy = true;
     private boolean manageDataDfsDirs = true;
     private StartupOption option = null;
+    private StartupOption dnOption = null;
     private String[] racks = null; 
     private String [] hosts = null;
     private long [] simulatedCapacities = null;
@@ -240,6 +241,14 @@ public class MiniDFSCluster {
       this.option = val;
       return this;
     }
+    
+    /**
+     * Default: null
+     */
+    public Builder dnStartupOption(StartupOption val) {
+      this.dnOption = val;
+      return this;
+    }
 
     /**
      * Default: null
@@ -356,6 +365,7 @@ public class MiniDFSCluster {
                        builder.enableManagedDfsDirsRedundancy,
                        builder.manageDataDfsDirs,
                        builder.option,
+                       builder.dnOption,
                        builder.racks,
                        builder.hosts,
                        builder.simulatedCapacities,
@@ -405,18 +415,24 @@ public class MiniDFSCluster {
   /**
    * Stores the information related to a namenode in the cluster
    */
-  static class NameNodeInfo {
+  public static class NameNodeInfo {
     final NameNode nameNode;
     final Configuration conf;
     final String nameserviceId;
     final String nnId;
+    StartupOption startOpt;
     NameNodeInfo(NameNode nn, String nameserviceId, String nnId,
-        Configuration conf) {
+        StartupOption startOpt, Configuration conf) {
       this.nameNode = nn;
       this.nameserviceId = nameserviceId;
       this.nnId = nnId;
+      this.startOpt = startOpt;
       this.conf = conf;
     }
+    
+    public void setStartOpt(StartupOption startOpt) {
+      this.startOpt = startOpt;
+    }
   }
   
   /**
@@ -602,8 +618,8 @@ public class MiniDFSCluster {
                         long[] simulatedCapacities) throws IOException {
     this.nameNodes = new NameNodeInfo[1]; // Single namenode in the cluster
     initMiniDFSCluster(conf, numDataNodes, StorageType.DEFAULT, format,
-        manageNameDfsDirs, true, manageDataDfsDirs, manageDataDfsDirs,
-        operation, racks, hosts,
+        manageNameDfsDirs, true, manageDataDfsDirs, manageDataDfsDirs, 
+        operation, null, racks, hosts,
         simulatedCapacities, null, true, false,
         MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0), true, false, false);
   }
@@ -612,7 +628,8 @@ public class MiniDFSCluster {
       Configuration conf,
       int numDataNodes, StorageType storageType, boolean format, boolean manageNameDfsDirs,
       boolean manageNameDfsSharedDirs, boolean enableManagedDfsDirsRedundancy,
-      boolean manageDataDfsDirs, StartupOption operation, String[] racks,
+      boolean manageDataDfsDirs, StartupOption startOpt,
+      StartupOption dnStartOpt, String[] racks,
       String[] hosts, long[] simulatedCapacities, String clusterId,
       boolean waitSafeMode, boolean setupHostsFile,
       MiniDFSNNTopology nnTopology, boolean checkExitOnShutdown,
@@ -661,7 +678,7 @@ public class MiniDFSCluster {
       createNameNodesAndSetConf(
           nnTopology, manageNameDfsDirs, manageNameDfsSharedDirs,
           enableManagedDfsDirsRedundancy,
-          format, operation, clusterId, conf);
+          format, startOpt, clusterId, conf);
     } catch (IOException ioe) {
       LOG.error("IOE creating namenodes. Permissions dump:\n" +
           createPermissionsDiagnosisString(data_dir));
@@ -674,13 +691,15 @@ public class MiniDFSCluster {
       }
     }
     
-    if (operation == StartupOption.RECOVER) {
+    if (startOpt == StartupOption.RECOVER) {
       return;
     }
 
     // Start the DataNodes
-    startDataNodes(conf, numDataNodes, storageType, manageDataDfsDirs, operation, racks,
-        hosts, simulatedCapacities, setupHostsFile, checkDataNodeAddrConfig, checkDataNodeHostConfig);
+    startDataNodes(conf, numDataNodes, storageType, manageDataDfsDirs,
+        dnStartOpt != null ? dnStartOpt : startOpt,
+        racks, hosts, simulatedCapacities, setupHostsFile,
+        checkDataNodeAddrConfig, checkDataNodeHostConfig);
     waitClusterUp();
     //make sure ProxyUsers uses the latest conf
     ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
@@ -758,6 +777,8 @@ public class MiniDFSCluster {
         if (manageNameDfsSharedDirs) {
           URI sharedEditsUri = getSharedEditsDir(nnCounter, nnCounter+nnIds.size()-1); 
           conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY, sharedEditsUri.toString());
+          // Clean out the shared edits dir completely, including all subdirectories.
+          FileUtil.fullyDelete(new File(sharedEditsUri));
         }
       }
 
@@ -857,7 +878,8 @@ public class MiniDFSCluster {
     URI srcDir = Lists.newArrayList(srcDirs).get(0);
     FileSystem dstFS = FileSystem.getLocal(dstConf).getRaw();
     for (URI dstDir : dstDirs) {
-      Preconditions.checkArgument(!dstDir.equals(srcDir));
+      Preconditions.checkArgument(!dstDir.equals(srcDir),
+          "src and dst are the same: " + dstDir);
       File dstDirF = new File(dstDir);
       if (dstDirF.exists()) {
         if (!FileUtil.fullyDelete(dstDirF)) {
@@ -891,6 +913,14 @@ public class MiniDFSCluster {
     conf.set(key, "127.0.0.1:" + nnConf.getIpcPort());
   }
   
+  private static String[] createArgs(StartupOption operation) {
+    String[] args = (operation == null ||
+        operation == StartupOption.FORMAT ||
+        operation == StartupOption.REGULAR) ?
+            new String[] {} : new String[] {operation.getName()};
+    return args;
+  }
+  
   private void createNameNode(int nnIndex, Configuration conf,
       int numDataNodes, boolean format, StartupOption operation,
       String clusterId, String nameserviceId,
@@ -905,10 +935,7 @@ public class MiniDFSCluster {
     }
     
     // Start the NameNode
-    String[] args = (operation == null ||
-                     operation == StartupOption.FORMAT ||
-                     operation == StartupOption.REGULAR) ?
-      new String[] {} : new String[] {operation.getName()};
+    String[] args = createArgs(operation);
     NameNode nn =  NameNode.createNameNode(args, conf);
     if (operation == StartupOption.RECOVER) {
       return;
@@ -930,7 +957,7 @@ public class MiniDFSCluster {
     DFSUtil.setGenericConf(conf, nameserviceId, nnId,
         DFS_NAMENODE_HTTP_ADDRESS_KEY);
     nameNodes[nnIndex] = new NameNodeInfo(nn, nameserviceId, nnId,
-        new Configuration(conf));
+        operation, new Configuration(conf));
   }
 
   /**
@@ -1498,7 +1525,7 @@ public class MiniDFSCluster {
       nn.stop();
       nn.join();
       Configuration conf = nameNodes[nnIndex].conf;
-      nameNodes[nnIndex] = new NameNodeInfo(null, null, null, conf);
+      nameNodes[nnIndex] = new NameNodeInfo(null, null, null, null, conf);
     }
   }
   
@@ -1552,10 +1579,17 @@ public class MiniDFSCluster {
       String... args) throws IOException {
     String nameserviceId = nameNodes[nnIndex].nameserviceId;
     String nnId = nameNodes[nnIndex].nnId;
+    StartupOption startOpt = nameNodes[nnIndex].startOpt;
     Configuration conf = nameNodes[nnIndex].conf;
     shutdownNameNode(nnIndex);
+    if (args.length != 0) {
+      startOpt = null;
+    } else {
+      args = createArgs(startOpt);
+    }
     NameNode nn = NameNode.createNameNode(args, conf);
-    nameNodes[nnIndex] = new NameNodeInfo(nn, nameserviceId, nnId, conf);
+    nameNodes[nnIndex] = new NameNodeInfo(nn, nameserviceId, nnId, startOpt,
+        conf);
     if (waitActive) {
       waitClusterUp();
       LOG.info("Restarted the namenode");

+ 20 - 25
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.After;
 import org.junit.Test;
@@ -97,10 +98,10 @@ public class TestDFSRollback {
    * Attempts to start a NameNode with the given operation.  Starting
    * the NameNode should throw an exception.
    */
-  void startNameNodeShouldFail(StartupOption operation, String searchString) {
+  void startNameNodeShouldFail(String searchString) {
     try {
+      NameNode.doRollback(conf, false);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
-                                                .startupOption(operation)
                                                 .format(false)
                                                 .manageDataDfsDirs(false)
                                                 .manageNameDfsDirs(false)
@@ -149,24 +150,19 @@ public class TestDFSRollback {
       log("Normal NameNode rollback", numDirs);
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
-      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
-                                                .format(false)
-                                                .manageDataDfsDirs(false)
-                                                .manageNameDfsDirs(false)
-                                                .startupOption(StartupOption.ROLLBACK)
-                                                .build();
+      NameNode.doRollback(conf, false);
       checkResult(NAME_NODE, nameNodeDirs);
-      cluster.shutdown();
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       
       log("Normal DataNode rollback", numDirs);
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
+      NameNode.doRollback(conf, false);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
                                                 .format(false)
                                                 .manageDataDfsDirs(false)
                                                 .manageNameDfsDirs(false)
-                                                .startupOption(StartupOption.ROLLBACK)
+                                                .dnStartupOption(StartupOption.ROLLBACK)
                                                 .build();
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "previous");
@@ -179,11 +175,12 @@ public class TestDFSRollback {
       log("Normal BlockPool rollback", numDirs);
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
+      NameNode.doRollback(conf, false);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
                                                 .format(false)
                                                 .manageDataDfsDirs(false)
                                                 .manageNameDfsDirs(false)
-                                                .startupOption(StartupOption.ROLLBACK)
+                                                .dnStartupOption(StartupOption.ROLLBACK)
                                                 .build();
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
       UpgradeUtilities.createBlockPoolStorageDirs(dataNodeDirs, "current",
@@ -217,10 +214,10 @@ public class TestDFSRollback {
       cluster.shutdown();
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       UpgradeUtilities.createEmptyDirs(dataNodeDirs);
-
+      
       log("NameNode rollback without existing previous dir", numDirs);
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
-      startNameNodeShouldFail(StartupOption.ROLLBACK,
+      startNameNodeShouldFail(
           "None of the storage directories contain previous fs state");
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       
@@ -237,15 +234,16 @@ public class TestDFSRollback {
       cluster.shutdown();
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       UpgradeUtilities.createEmptyDirs(dataNodeDirs);
-
+      
       log("DataNode rollback with future stored layout version in previous", numDirs);
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
+      NameNode.doRollback(conf, false);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
                                                 .format(false)
                                                 .manageDataDfsDirs(false)
                                                 .manageNameDfsDirs(false)
-                                                .startupOption(StartupOption.ROLLBACK)
+                                                .dnStartupOption(StartupOption.ROLLBACK)
                                                 .build();
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
       baseDirs = UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "previous");
@@ -266,11 +264,12 @@ public class TestDFSRollback {
       log("DataNode rollback with newer fsscTime in previous", numDirs);
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
+      NameNode.doRollback(conf, false);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
                                                 .format(false)
                                                 .manageDataDfsDirs(false)
                                                 .manageNameDfsDirs(false)
-                                                .startupOption(StartupOption.ROLLBACK)
+                                                .dnStartupOption(StartupOption.ROLLBACK)
                                                 .build();
       
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
@@ -287,21 +286,19 @@ public class TestDFSRollback {
       cluster.shutdown();
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       UpgradeUtilities.createEmptyDirs(dataNodeDirs);
-
+      
       log("NameNode rollback with no edits file", numDirs);
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       baseDirs = UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
       deleteMatchingFiles(baseDirs, "edits.*");
-      startNameNodeShouldFail(StartupOption.ROLLBACK,
-          "Gap in transactions");
+      startNameNodeShouldFail("Gap in transactions");
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       
       log("NameNode rollback with no image file", numDirs);
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       baseDirs = UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
       deleteMatchingFiles(baseDirs, "fsimage_.*");
-      startNameNodeShouldFail(StartupOption.ROLLBACK,
-          "No valid image files found");
+      startNameNodeShouldFail("No valid image files found");
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       
       log("NameNode rollback with corrupt version file", numDirs);
@@ -313,8 +310,7 @@ public class TestDFSRollback {
             "layoutVersion".getBytes(Charsets.UTF_8),
             "xxxxxxxxxxxxx".getBytes(Charsets.UTF_8));
       }
-      startNameNodeShouldFail(StartupOption.ROLLBACK,
-          "file VERSION has layoutVersion missing");
+      startNameNodeShouldFail("file VERSION has layoutVersion missing");
 
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       
@@ -328,8 +324,7 @@ public class TestDFSRollback {
       
       UpgradeUtilities.createNameNodeVersionFile(conf, baseDirs,
           storageInfo, UpgradeUtilities.getCurrentBlockPoolID(cluster));
-      startNameNodeShouldFail(StartupOption.ROLLBACK,
-          "Cannot rollback to storage version 1 using this version");
+      startNameNodeShouldFail("Cannot rollback to storage version 1 using this version");
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
     } // end numDir loop
   }

+ 34 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java

@@ -62,6 +62,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Shell;
 import org.junit.Assume;
 import org.junit.Before;
@@ -764,4 +765,37 @@ public class TestDFSUtil {
     assertEquals(4*24*60*60*1000l, DFSUtil.parseRelativeTime("4d"));
     assertEquals(999*24*60*60*1000l, DFSUtil.parseRelativeTime("999d"));
   }
+  
+  @Test
+  public void testAssertAllResultsEqual() {
+    checkAllResults(new Long[]{}, true);
+    checkAllResults(new Long[]{1l}, true);
+    checkAllResults(new Long[]{1l, 1l}, true);
+    checkAllResults(new Long[]{1l, 1l, 1l}, true);
+    checkAllResults(new Long[]{new Long(1), new Long(1)}, true);
+    checkAllResults(new Long[]{null, null, null}, true);
+    
+    checkAllResults(new Long[]{1l, 2l}, false);
+    checkAllResults(new Long[]{2l, 1l}, false);
+    checkAllResults(new Long[]{1l, 2l, 1l}, false);
+    checkAllResults(new Long[]{2l, 1l, 1l}, false);
+    checkAllResults(new Long[]{1l, 1l, 2l}, false);
+    checkAllResults(new Long[]{1l, null}, false);
+    checkAllResults(new Long[]{null, 1l}, false);
+    checkAllResults(new Long[]{1l, null, 1l}, false);
+  }
+  
+  private static void checkAllResults(Long[] toCheck, boolean shouldSucceed) {
+    if (shouldSucceed) {
+      DFSUtil.assertAllResultsEqual(Arrays.asList(toCheck));
+    } else {
+      try {
+        DFSUtil.assertAllResultsEqual(Arrays.asList(toCheck));
+        fail("Should not have succeeded with input: " +
+            Arrays.toString(toCheck));
+      } catch (AssertionError ae) {
+        GenericTestUtils.assertExceptionContains("Not all elements match", ae);
+      }
+    }
+  }
 }

+ 59 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java

@@ -19,20 +19,28 @@ package org.apache.hadoop.hdfs;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.io.File;
 import java.io.IOException;
+import java.io.RandomAccessFile;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.TestInterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.Test;
 
 public class TestLeaseRecovery {
@@ -148,4 +156,55 @@ public class TestLeaseRecovery {
       if (cluster != null) {cluster.shutdown();}
     }
   }
+
+  /**
+   * Block Recovery when the meta file not having crcs for all chunks in block
+   * file
+   */
+  @Test
+  public void testBlockRecoveryWithLessMetafile() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
+        UserGroupInformation.getCurrentUser().getShortUserName());
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
+        .build();
+    Path file = new Path("/testRecoveryFile");
+    DistributedFileSystem dfs = cluster.getFileSystem();
+    FSDataOutputStream out = dfs.create(file);
+    int count = 0;
+    while (count < 2 * 1024 * 1024) {
+      out.writeBytes("Data");
+      count += 4;
+    }
+    out.hsync();
+    // abort the original stream
+    ((DFSOutputStream) out.getWrappedStream()).abort();
+
+    LocatedBlocks locations = cluster.getNameNodeRpc().getBlockLocations(
+        file.toString(), 0, count);
+    ExtendedBlock block = locations.get(0).getBlock();
+    DataNode dn = cluster.getDataNodes().get(0);
+    BlockLocalPathInfo localPathInfo = dn.getBlockLocalPathInfo(block, null);
+    File metafile = new File(localPathInfo.getMetaPath());
+    assertTrue(metafile.exists());
+
+    // reduce the block meta file size
+    RandomAccessFile raf = new RandomAccessFile(metafile, "rw");
+    raf.setLength(metafile.length() - 20);
+    raf.close();
+
+    // restart DN to make replica to RWR
+    DataNodeProperties dnProp = cluster.stopDataNode(0);
+    cluster.restartDataNode(dnProp, true);
+
+    // try to recover the lease
+    DistributedFileSystem newdfs = (DistributedFileSystem) FileSystem
+        .newInstance(cluster.getConfiguration(0));
+    count = 0;
+    while (++count < 10 && !newdfs.recoverLease(file)) {
+      Thread.sleep(1000);
+    }
+    assertTrue("File should be closed", newdfs.recoverLease(file));
+
+  }
 }

+ 9 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniJournalCluster.java

@@ -167,8 +167,16 @@ public class MiniJournalCluster {
     return new File(baseDir, "journalnode-" + idx).getAbsoluteFile();
   }
   
+  public File getJournalDir(int idx, String jid) {
+    return new File(getStorageDir(idx), jid);
+  }
+  
   public File getCurrentDir(int idx, String jid) {
-    return new File(new File(getStorageDir(idx), jid), "current");
+    return new File(getJournalDir(idx, jid), "current");
+  }
+  
+  public File getPreviousDir(int idx, String jid) {
+    return new File(getJournalDir(idx, jid), "previous");
   }
 
   public JournalNode getJournalNode(int i) {

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
 
@@ -47,6 +48,7 @@ public class MiniQJMHACluster {
 
   public static class Builder {
     private final Configuration conf;
+    private StartupOption startOpt = null;
     private final MiniDFSCluster.Builder dfsBuilder;
     
     public Builder(Configuration conf) {
@@ -61,6 +63,10 @@ public class MiniQJMHACluster {
     public MiniQJMHACluster build() throws IOException {
       return new MiniQJMHACluster(this);
     }
+
+    public void startupOption(StartupOption startOpt) {
+      this.startOpt = startOpt;
+    }
   }
   
   public static MiniDFSNNTopology createDefaultTopology() {
@@ -95,6 +101,9 @@ public class MiniQJMHACluster {
     Configuration confNN0 = cluster.getConfiguration(0);
     NameNode.initializeSharedEdits(confNN0, true);
     
+    cluster.getNameNodeInfos()[0].setStartOpt(builder.startOpt);
+    cluster.getNameNodeInfos()[1].setStartOpt(builder.startOpt);
+    
     // restart the cluster
     cluster.restartNameNodes();
   }

+ 44 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java

@@ -18,16 +18,17 @@
 
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
 
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
 import java.util.regex.Pattern;
 
 import org.apache.commons.logging.impl.Log4JLogger;
@@ -46,6 +47,8 @@ import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.PathUtils;
+import org.apache.log4j.Appender;
+import org.apache.log4j.AsyncAppender;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -54,13 +57,30 @@ import org.apache.log4j.RollingFileAppender;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
 /**
  * A JUnit test that audit logs are generated
  */
+@RunWith(Parameterized.class)
 public class TestAuditLogs {
   static final String auditLogFile = PathUtils.getTestDirName(TestAuditLogs.class) + "/TestAuditLogs-audit.log";
+  boolean useAsyncLog;
   
+  @Parameters
+  public static Collection<Object[]> data() {
+    Collection<Object[]> params = new ArrayList<Object[]>();
+    params.add(new Object[]{new Boolean(false)});
+    params.add(new Object[]{new Boolean(true)});
+    return params;
+  }
+  
+  public TestAuditLogs(boolean useAsyncLog) {
+    this.useAsyncLog = useAsyncLog;
+  }
+
   // Pattern for: 
   // allowed=(true|false) ugi=name ip=/address cmd={cmd} src={path} dst=null perm=null
   static final Pattern auditPattern = Pattern.compile(
@@ -84,17 +104,28 @@ public class TestAuditLogs {
 
   @Before
   public void setupCluster() throws Exception {
+    // must configure prior to instantiating the namesystem because it
+    // will reconfigure the logger if async is enabled
+    configureAuditLogs();
     conf = new HdfsConfiguration();
     final long precision = 1L;
     conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, precision);
     conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
     conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY, useAsyncLog);
     util = new DFSTestUtil.Builder().setName("TestAuditAllowed").
         setNumFiles(20).build();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
     fs = cluster.getFileSystem();
     util.createFiles(fs, fileName);
 
+    // make sure the appender is what it's supposed to be
+    Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger();
+    @SuppressWarnings("unchecked")
+    List<Appender> appenders = Collections.list(logger.getAllAppenders());
+    assertEquals(1, appenders.size());
+    assertEquals(useAsyncLog, appenders.get(0) instanceof AsyncAppender);
+    
     fnames = util.getFileNames(fileName);
     util.waitReplication(fs, fileName, (short)3);
     userGroupInfo = UserGroupInformation.createUserForTesting(username, groups);
@@ -203,6 +234,7 @@ public class TestAuditLogs {
     try {
       hftpFs = (HftpFileSystem) new Path(hftpUri).getFileSystem(conf);
       InputStream istream = hftpFs.open(file);
+      @SuppressWarnings("unused")
       int val = istream.read();
       istream.close();
 
@@ -234,6 +266,12 @@ public class TestAuditLogs {
 
   /** Sets up log4j logger for auditlogs */
   private void setupAuditLogs() throws IOException {
+    Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger();
+    // enable logging now that the test is ready to run
+    logger.setLevel(Level.INFO);
+  }
+  
+  private void configureAuditLogs() throws IOException {
     // Shutdown the LogManager to release all logger open file handles.
     // Unfortunately, Apache commons logging library does not provide
     // means to release underlying loggers. For additional info look up
@@ -245,7 +283,8 @@ public class TestAuditLogs {
       assertTrue(file.delete());
     }
     Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger();
-    logger.setLevel(Level.INFO);
+    // disable logging while the cluster startup preps files
+    logger.setLevel(Level.OFF);
     PatternLayout layout = new PatternLayout("%m%n");
     RollingFileAppender appender = new RollingFileAppender(layout, auditLogFile);
     logger.addAppender(appender);

+ 25 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java

@@ -27,6 +27,8 @@ import java.util.Collection;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.junit.Test;
 
@@ -191,6 +193,29 @@ public class TestGenericJournalConf {
       shouldPromptCalled = true;
       return false;
     }
+
+    @Override
+    public void doPreUpgrade() throws IOException {}
+
+    @Override
+    public void doUpgrade(Storage storage) throws IOException {}
+
+    @Override
+    public void doFinalize() throws IOException {}
+
+    @Override
+    public boolean canRollBack(StorageInfo storage, StorageInfo prevStorage, int targetLayoutVersion)
+        throws IOException {
+      return false;
+    }
+
+    @Override
+    public void doRollback() throws IOException {}
+
+    @Override
+    public long getJournalCTime() throws IOException {
+      return -1;
+    }
   }
 
   public static class BadConstructorJournalManager extends DummyJournalManager {

+ 71 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -26,6 +26,7 @@ import static org.junit.Assert.fail;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
@@ -952,7 +953,76 @@ public class TestINodeFile {
       }
     }
   }
-  
+  @Test
+  public void testLocationLimitInListingOps() throws Exception {
+    final Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_LIST_LIMIT, 9); // 3 blocks * 3 replicas
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+      cluster.waitActive();
+      final DistributedFileSystem hdfs = cluster.getFileSystem();
+      ArrayList<String> source = new ArrayList<String>();
+
+      // tmp1 holds files with 3 blocks, 3 replicas
+      // tmp2 holds files with 3 blocks, 1 replica
+      hdfs.mkdirs(new Path("/tmp1"));
+      hdfs.mkdirs(new Path("/tmp2"));
+
+      source.add("f1");
+      source.add("f2");
+
+      int numEntries = source.size();
+      for (int j=0;j<numEntries;j++) {
+          DFSTestUtil.createFile(hdfs, new Path("/tmp1/"+source.get(j)), 4096,
+          3*1024-100, 1024, (short) 3, 0);
+      }
+
+      byte[] start = HdfsFileStatus.EMPTY_NAME;
+      for (int j=0;j<numEntries;j++) {
+          DirectoryListing dl = cluster.getNameNodeRpc().getListing("/tmp1",
+              start, true);
+          assertTrue(dl.getPartialListing().length == 1);
+          for (int i=0;i<dl.getPartialListing().length; i++) {
+              source.remove(dl.getPartialListing()[i].getLocalName());
+          }
+          start = dl.getLastName();
+      }
+      // Verify we have listed all entries in the directory.
+      assertTrue(source.size() == 0);
+
+      // Now create 6 files, each with 3 locations. Should take 2 iterations of 3
+      source.add("f1");
+      source.add("f2");
+      source.add("f3");
+      source.add("f4");
+      source.add("f5");
+      source.add("f6");
+      numEntries = source.size();
+      for (int j=0;j<numEntries;j++) {
+          DFSTestUtil.createFile(hdfs, new Path("/tmp2/"+source.get(j)), 4096,
+          3*1024-100, 1024, (short) 1, 0);
+      }
+
+      start = HdfsFileStatus.EMPTY_NAME;
+      for (int j=0;j<numEntries/3;j++) {
+          DirectoryListing dl = cluster.getNameNodeRpc().getListing("/tmp2",
+              start, true);
+          assertTrue(dl.getPartialListing().length == 3);
+          for (int i=0;i<dl.getPartialListing().length; i++) {
+              source.remove(dl.getPartialListing()[i].getLocalName());
+          }
+          start = dl.getLastName();
+      }
+      // Verify we have listed all entries in tmp2.
+      assertTrue(source.size() == 0);
+  } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
   @Test
   public void testFilesInGetListingOps() throws Exception {
     final Configuration conf = new Configuration();

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

@@ -91,7 +91,7 @@ public class TestBootstrapStandby {
       fail("Did not throw");
     } catch (IOException ioe) {
       GenericTestUtils.assertExceptionContains(
-          "Cannot start an HA namenode with name dirs that need recovery",
+          "storage directory does not exist or is not accessible",
           ioe);
     }
     

+ 674 - 49
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java

@@ -1,89 +1,506 @@
 /**
-* 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.
-*/
+ * 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.ha;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;
 import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Collection;
 
 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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
+import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster.Builder;
+import org.apache.hadoop.hdfs.qjournal.server.Journal;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.tools.DFSAdmin;
+import org.apache.hadoop.hdfs.util.PersistentLongFile;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Before;
 import org.junit.Test;
 
-import com.google.common.collect.Lists;
+import com.google.common.base.Joiner;
 
 /**
  * Tests for upgrading with HA enabled.
  */
 public class TestDFSUpgradeWithHA {
-  
+
   private static final Log LOG = LogFactory.getLog(TestDFSUpgradeWithHA.class);
+  
+  private Configuration conf;
+  
+  @Before
+  public void createConfiguration() {
+    conf = new HdfsConfiguration();
+    // Turn off persistent IPC, so that the DFSClient can survive NN restart
+    conf.setInt(
+        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
+        0);
+  }
+
+  private static void assertCTimesEqual(MiniDFSCluster cluster) {
+    long nn1CTime = cluster.getNamesystem(0).getFSImage().getStorage().getCTime();
+    long nn2CTime = cluster.getNamesystem(1).getFSImage().getStorage().getCTime();
+    assertEquals(nn1CTime, nn2CTime);
+  }
+
+  private static void checkClusterPreviousDirExistence(MiniDFSCluster cluster,
+      boolean shouldExist) {
+    for (int i = 0; i < 2; i++) {
+      checkNnPreviousDirExistence(cluster, i, shouldExist);
+    }
+  }
+
+  private static void checkNnPreviousDirExistence(MiniDFSCluster cluster,
+      int index, boolean shouldExist) {
+    Collection<URI> nameDirs = cluster.getNameDirs(index);
+    for (URI nnDir : nameDirs) {
+      checkPreviousDirExistence(new File(nnDir), shouldExist);
+    }
+  }
 
+  private static void checkJnPreviousDirExistence(MiniQJMHACluster jnCluster,
+      boolean shouldExist) throws IOException {
+    for (int i = 0; i < 3; i++) {
+      checkPreviousDirExistence(
+          jnCluster.getJournalCluster().getJournalDir(i, "ns1"), shouldExist);
+    }
+    if (shouldExist) {
+      assertEpochFilesCopied(jnCluster);
+    }
+  }
+
+  private static void assertEpochFilesCopied(MiniQJMHACluster jnCluster)
+      throws IOException {
+    for (int i = 0; i < 3; i++) {
+      File journalDir = jnCluster.getJournalCluster().getJournalDir(i, "ns1");
+      File currDir = new File(journalDir, "current");
+      File prevDir = new File(journalDir, "previous");
+      for (String fileName : new String[]{ Journal.LAST_PROMISED_FILENAME,
+          Journal.LAST_WRITER_EPOCH }) {
+        File prevFile = new File(prevDir, fileName);
+        // Possible the prev file doesn't exist, e.g. if there has never been a
+        // writer before the upgrade.
+        if (prevFile.exists()) {
+          PersistentLongFile prevLongFile = new PersistentLongFile(prevFile, -10);
+          PersistentLongFile currLongFile = new PersistentLongFile(new File(currDir,
+              fileName), -11);
+          assertTrue("Value in " + fileName + " has decreased on upgrade in "
+              + journalDir, prevLongFile.get() <= currLongFile.get());
+        }
+      }
+    }
+  }
+
+  private static void checkPreviousDirExistence(File rootDir,
+      boolean shouldExist) {
+    File previousDir = new File(rootDir, "previous");
+    if (shouldExist) {
+      assertTrue(previousDir + " does not exist", previousDir.exists());
+    } else {
+      assertFalse(previousDir + " does exist", previousDir.exists());
+    }
+  }
+  
+  private void runFinalizeCommand(MiniDFSCluster cluster)
+      throws IOException {
+    HATestUtil.setFailoverConfigurations(cluster, conf);
+    new DFSAdmin(conf).finalizeUpgrade();
+  }
+  
   /**
-   * Make sure that an HA NN refuses to start if given an upgrade-related
-   * startup option.
+   * Ensure that an admin cannot finalize an HA upgrade without at least one NN
+   * being active.
    */
   @Test
-  public void testStartingWithUpgradeOptionsFails() throws IOException {
-    for (StartupOption startOpt : Lists.newArrayList(new StartupOption[] {
-        StartupOption.UPGRADE, StartupOption.FINALIZE,
-        StartupOption.ROLLBACK })) {
-      MiniDFSCluster cluster = null;
+  public void testCannotFinalizeIfNoActive() throws IOException,
+      URISyntaxException {
+    MiniDFSCluster cluster = null;
+    FileSystem fs = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf)
+          .nnTopology(MiniDFSNNTopology.simpleHATopology())
+          .numDataNodes(0)
+          .build();
+
+      File sharedDir = new File(cluster.getSharedEditsDir(0, 1));
+      
+      // No upgrade is in progress at the moment.
+      checkClusterPreviousDirExistence(cluster, false);
+      assertCTimesEqual(cluster);
+      checkPreviousDirExistence(sharedDir, false);
+      
+      // Transition NN0 to active and do some FS ops.
+      cluster.transitionToActive(0);
+      fs = HATestUtil.configureFailoverFs(cluster, conf);
+      assertTrue(fs.mkdirs(new Path("/foo1")));
+      
+      // Do the upgrade. Shut down NN1 and then restart NN0 with the upgrade
+      // flag.
+      cluster.shutdownNameNode(1);
+      cluster.getNameNodeInfos()[0].setStartOpt(StartupOption.UPGRADE);
+      cluster.restartNameNode(0, false);
+      
+      checkNnPreviousDirExistence(cluster, 0, true);
+      checkNnPreviousDirExistence(cluster, 1, false);
+      checkPreviousDirExistence(sharedDir, true);
+      
+      // NN0 should come up in the active state when given the -upgrade option,
+      // so no need to transition it to active.
+      assertTrue(fs.mkdirs(new Path("/foo2")));
+      
+      // Restart NN0 without the -upgrade flag, to make sure that works.
+      cluster.getNameNodeInfos()[0].setStartOpt(StartupOption.REGULAR);
+      cluster.restartNameNode(0, false);
+      
+      // Make sure we can still do FS ops after upgrading.
+      cluster.transitionToActive(0);
+      assertTrue(fs.mkdirs(new Path("/foo3")));
+      
+      // Now bootstrap the standby with the upgraded info.
+      int rc = BootstrapStandby.run(
+          new String[]{"-force"},
+          cluster.getConfiguration(1));
+      assertEquals(0, rc);
+      
+      // Now restart NN1 and make sure that we can do ops against that as well.
+      cluster.restartNameNode(1);
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(1);
+      assertTrue(fs.mkdirs(new Path("/foo4")));
+      
+      assertCTimesEqual(cluster);
+      
+      // Now there's no active NN.
+      cluster.transitionToStandby(1);
+
       try {
-        cluster = new MiniDFSCluster.Builder(new Configuration())
-            .nnTopology(MiniDFSNNTopology.simpleHATopology())
-            .startupOption(startOpt)
-            .numDataNodes(0)
-            .build();
-        fail("Should not have been able to start an HA NN in upgrade mode");
-      } catch (IllegalArgumentException iae) {
+        runFinalizeCommand(cluster);
+        fail("Should not have been able to finalize upgrade with no NN active");
+      } catch (IOException ioe) {
         GenericTestUtils.assertExceptionContains(
-            "Cannot perform DFS upgrade with HA enabled.", iae);
-        LOG.info("Got expected exception", iae);
-      } finally {
-        if (cluster != null) {
-          cluster.shutdown();
-        }
+            "Cannot finalize with no NameNode active", ioe);
+      }
+    } finally {
+      if (fs != null) {
+        fs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
       }
     }
   }
-  
+
   /**
-   * Make sure that an HA NN won't start if a previous upgrade was in progress.
+   * Make sure that an HA NN with NFS-based HA can successfully start and
+   * upgrade.
    */
   @Test
-  public void testStartingWithUpgradeInProgressFails() throws Exception {
+  public void testNfsUpgrade() throws IOException, URISyntaxException {
     MiniDFSCluster cluster = null;
+    FileSystem fs = null;
     try {
-      cluster = new MiniDFSCluster.Builder(new Configuration())
+      cluster = new MiniDFSCluster.Builder(conf)
           .nnTopology(MiniDFSNNTopology.simpleHATopology())
           .numDataNodes(0)
           .build();
       
+      File sharedDir = new File(cluster.getSharedEditsDir(0, 1));
+      
+      // No upgrade is in progress at the moment.
+      checkClusterPreviousDirExistence(cluster, false);
+      assertCTimesEqual(cluster);
+      checkPreviousDirExistence(sharedDir, false);
+      
+      // Transition NN0 to active and do some FS ops.
+      cluster.transitionToActive(0);
+      fs = HATestUtil.configureFailoverFs(cluster, conf);
+      assertTrue(fs.mkdirs(new Path("/foo1")));
+      
+      // Do the upgrade. Shut down NN1 and then restart NN0 with the upgrade
+      // flag.
+      cluster.shutdownNameNode(1);
+      cluster.getNameNodeInfos()[0].setStartOpt(StartupOption.UPGRADE);
+      cluster.restartNameNode(0, false);
+      
+      checkNnPreviousDirExistence(cluster, 0, true);
+      checkNnPreviousDirExistence(cluster, 1, false);
+      checkPreviousDirExistence(sharedDir, true);
+      
+      // NN0 should come up in the active state when given the -upgrade option,
+      // so no need to transition it to active.
+      assertTrue(fs.mkdirs(new Path("/foo2")));
+      
+      // Restart NN0 without the -upgrade flag, to make sure that works.
+      cluster.getNameNodeInfos()[0].setStartOpt(StartupOption.REGULAR);
+      cluster.restartNameNode(0, false);
+      
+      // Make sure we can still do FS ops after upgrading.
+      cluster.transitionToActive(0);
+      assertTrue(fs.mkdirs(new Path("/foo3")));
+      
+      // Now bootstrap the standby with the upgraded info.
+      int rc = BootstrapStandby.run(
+          new String[]{"-force"},
+          cluster.getConfiguration(1));
+      assertEquals(0, rc);
+      
+      // Now restart NN1 and make sure that we can do ops against that as well.
+      cluster.restartNameNode(1);
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(1);
+      assertTrue(fs.mkdirs(new Path("/foo4")));
+      
+      assertCTimesEqual(cluster);
+    } finally {
+      if (fs != null) {
+        fs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Make sure that an HA NN can successfully upgrade when configured using
+   * JournalNodes.
+   */
+  @Test
+  public void testUpgradeWithJournalNodes() throws IOException,
+      URISyntaxException {
+    MiniQJMHACluster qjCluster = null;
+    FileSystem fs = null;
+    try {
+      Builder builder = new MiniQJMHACluster.Builder(conf);
+      builder.getDfsBuilder()
+          .numDataNodes(0);
+      qjCluster = builder.build();
+
+      MiniDFSCluster cluster = qjCluster.getDfsCluster();
+      
+      // No upgrade is in progress at the moment.
+      checkJnPreviousDirExistence(qjCluster, false);
+      checkClusterPreviousDirExistence(cluster, false);
+      assertCTimesEqual(cluster);
+      
+      // Transition NN0 to active and do some FS ops.
+      cluster.transitionToActive(0);
+      fs = HATestUtil.configureFailoverFs(cluster, conf);
+      assertTrue(fs.mkdirs(new Path("/foo1")));
+      
+      // Do the upgrade. Shut down NN1 and then restart NN0 with the upgrade
+      // flag.
+      cluster.shutdownNameNode(1);
+      cluster.getNameNodeInfos()[0].setStartOpt(StartupOption.UPGRADE);
+      cluster.restartNameNode(0, false);
+      
+      checkNnPreviousDirExistence(cluster, 0, true);
+      checkNnPreviousDirExistence(cluster, 1, false);
+      checkJnPreviousDirExistence(qjCluster, true);
+      
+      // NN0 should come up in the active state when given the -upgrade option,
+      // so no need to transition it to active.
+      assertTrue(fs.mkdirs(new Path("/foo2")));
+      
+      // Restart NN0 without the -upgrade flag, to make sure that works.
+      cluster.getNameNodeInfos()[0].setStartOpt(StartupOption.REGULAR);
+      cluster.restartNameNode(0, false);
+      
+      // Make sure we can still do FS ops after upgrading.
+      cluster.transitionToActive(0);
+      assertTrue(fs.mkdirs(new Path("/foo3")));
+      
+      // Now bootstrap the standby with the upgraded info.
+      int rc = BootstrapStandby.run(
+          new String[]{"-force"},
+          cluster.getConfiguration(1));
+      assertEquals(0, rc);
+      
+      // Now restart NN1 and make sure that we can do ops against that as well.
+      cluster.restartNameNode(1);
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(1);
+      assertTrue(fs.mkdirs(new Path("/foo4")));
+      
+      assertCTimesEqual(cluster);
+    } finally {
+      if (fs != null) {
+        fs.close();
+      }
+      if (qjCluster != null) {
+        qjCluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testFinalizeWithJournalNodes() throws IOException,
+      URISyntaxException {
+    MiniQJMHACluster qjCluster = null;
+    FileSystem fs = null;
+    try {
+      Builder builder = new MiniQJMHACluster.Builder(conf);
+      builder.getDfsBuilder()
+          .numDataNodes(0);
+      qjCluster = builder.build();
+
+      MiniDFSCluster cluster = qjCluster.getDfsCluster();
+      
+      // No upgrade is in progress at the moment.
+      checkJnPreviousDirExistence(qjCluster, false);
+      checkClusterPreviousDirExistence(cluster, false);
+      assertCTimesEqual(cluster);
+      
+      // Transition NN0 to active and do some FS ops.
+      cluster.transitionToActive(0);
+      fs = HATestUtil.configureFailoverFs(cluster, conf);
+      assertTrue(fs.mkdirs(new Path("/foo1")));
+      
+      // Do the upgrade. Shut down NN1 and then restart NN0 with the upgrade
+      // flag.
+      cluster.shutdownNameNode(1);
+      cluster.getNameNodeInfos()[0].setStartOpt(StartupOption.UPGRADE);
+      cluster.restartNameNode(0, false);
+      
+      assertTrue(fs.mkdirs(new Path("/foo2")));
+      
+      checkNnPreviousDirExistence(cluster, 0, true);
+      checkNnPreviousDirExistence(cluster, 1, false);
+      checkJnPreviousDirExistence(qjCluster, true);
+      
+      // Now bootstrap the standby with the upgraded info.
+      int rc = BootstrapStandby.run(
+          new String[]{"-force"},
+          cluster.getConfiguration(1));
+      assertEquals(0, rc);
+      
+      cluster.restartNameNode(1);
+      
+      runFinalizeCommand(cluster);
+      
+      checkClusterPreviousDirExistence(cluster, false);
+      checkJnPreviousDirExistence(qjCluster, false);
+      assertCTimesEqual(cluster);
+    } finally {
+      if (fs != null) {
+        fs.close();
+      }
+      if (qjCluster != null) {
+        qjCluster.shutdown();
+      }
+    }
+  }
+  
+  /**
+   * Make sure that even if the NN which initiated the upgrade is in the standby
+   * state that we're allowed to finalize.
+   */
+  @Test
+  public void testFinalizeFromSecondNameNodeWithJournalNodes()
+      throws IOException, URISyntaxException {
+    MiniQJMHACluster qjCluster = null;
+    FileSystem fs = null;
+    try {
+      Builder builder = new MiniQJMHACluster.Builder(conf);
+      builder.getDfsBuilder()
+          .numDataNodes(0);
+      qjCluster = builder.build();
+
+      MiniDFSCluster cluster = qjCluster.getDfsCluster();
+      
+      // No upgrade is in progress at the moment.
+      checkJnPreviousDirExistence(qjCluster, false);
+      checkClusterPreviousDirExistence(cluster, false);
+      assertCTimesEqual(cluster);
+      
+      // Transition NN0 to active and do some FS ops.
+      cluster.transitionToActive(0);
+      fs = HATestUtil.configureFailoverFs(cluster, conf);
+      assertTrue(fs.mkdirs(new Path("/foo1")));
+      
+      // Do the upgrade. Shut down NN1 and then restart NN0 with the upgrade
+      // flag.
+      cluster.shutdownNameNode(1);
+      cluster.getNameNodeInfos()[0].setStartOpt(StartupOption.UPGRADE);
+      cluster.restartNameNode(0, false);
+      
+      checkNnPreviousDirExistence(cluster, 0, true);
+      checkNnPreviousDirExistence(cluster, 1, false);
+      checkJnPreviousDirExistence(qjCluster, true);
+      
+      // Now bootstrap the standby with the upgraded info.
+      int rc = BootstrapStandby.run(
+          new String[]{"-force"},
+          cluster.getConfiguration(1));
+      assertEquals(0, rc);
+      
+      cluster.restartNameNode(1);
+      
+      // Make the second NN (not the one that initiated the upgrade) active when
+      // the finalize command is run.
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(1);
+      
+      runFinalizeCommand(cluster);
+      
+      checkClusterPreviousDirExistence(cluster, false);
+      checkJnPreviousDirExistence(qjCluster, false);
+      assertCTimesEqual(cluster);
+    } finally {
+      if (fs != null) {
+        fs.close();
+      }
+      if (qjCluster != null) {
+        qjCluster.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Make sure that an HA NN will start if a previous upgrade was in progress.
+   */
+  @Test
+  public void testStartingWithUpgradeInProgressSucceeds() throws Exception {
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf)
+          .nnTopology(MiniDFSNNTopology.simpleHATopology())
+          .numDataNodes(0)
+          .build();
+
       // Simulate an upgrade having started.
       for (int i = 0; i < 2; i++) {
         for (URI uri : cluster.getNameDirs(i)) {
@@ -92,18 +509,226 @@ public class TestDFSUpgradeWithHA {
           assertTrue(prevTmp.mkdirs());
         }
       }
-      
+
       cluster.restartNameNodes();
-      fail("Should not have been able to start an HA NN with an in-progress upgrade");
-    } catch (IOException ioe) {
-      GenericTestUtils.assertExceptionContains(
-          "Cannot start an HA namenode with name dirs that need recovery.",
-          ioe);
-      LOG.info("Got expected exception", ioe);
     } finally {
       if (cluster != null) {
         cluster.shutdown();
       }
     }
   }
+
+  /**
+   * Test rollback with NFS shared dir.
+   */
+  @Test
+  public void testRollbackWithNfs() throws Exception {
+    MiniDFSCluster cluster = null;
+    FileSystem fs = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf)
+          .nnTopology(MiniDFSNNTopology.simpleHATopology())
+          .numDataNodes(0)
+          .build();
+
+      File sharedDir = new File(cluster.getSharedEditsDir(0, 1));
+      
+      // No upgrade is in progress at the moment.
+      checkClusterPreviousDirExistence(cluster, false);
+      assertCTimesEqual(cluster);
+      checkPreviousDirExistence(sharedDir, false);
+      
+      // Transition NN0 to active and do some FS ops.
+      cluster.transitionToActive(0);
+      fs = HATestUtil.configureFailoverFs(cluster, conf);
+      assertTrue(fs.mkdirs(new Path("/foo1")));
+      
+      // Do the upgrade. Shut down NN1 and then restart NN0 with the upgrade
+      // flag.
+      cluster.shutdownNameNode(1);
+      cluster.getNameNodeInfos()[0].setStartOpt(StartupOption.UPGRADE);
+      cluster.restartNameNode(0, false);
+      
+      checkNnPreviousDirExistence(cluster, 0, true);
+      checkNnPreviousDirExistence(cluster, 1, false);
+      checkPreviousDirExistence(sharedDir, true);
+      
+      // NN0 should come up in the active state when given the -upgrade option,
+      // so no need to transition it to active.
+      assertTrue(fs.mkdirs(new Path("/foo2")));
+      
+      // Now bootstrap the standby with the upgraded info.
+      int rc = BootstrapStandby.run(
+          new String[]{"-force"},
+          cluster.getConfiguration(1));
+      assertEquals(0, rc);
+      
+      cluster.restartNameNode(1);
+      
+      checkNnPreviousDirExistence(cluster, 0, true);
+      checkNnPreviousDirExistence(cluster, 1, false);
+      checkPreviousDirExistence(sharedDir, true);
+      assertCTimesEqual(cluster);
+      
+      // Now shut down the cluster and do the rollback.
+      Collection<URI> nn1NameDirs = cluster.getNameDirs(0);
+      cluster.shutdown();
+
+      conf.setStrings(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, Joiner.on(",").join(nn1NameDirs));
+      NameNode.doRollback(conf, false);
+
+      // The rollback operation should have rolled back the first NN's local
+      // dirs, and the shared dir, but not the other NN's dirs. Those have to be
+      // done by bootstrapping the standby.
+      checkNnPreviousDirExistence(cluster, 0, false);
+      checkPreviousDirExistence(sharedDir, false);
+    } finally {
+      if (fs != null) {
+        fs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+  
+  @Test
+  public void testRollbackWithJournalNodes() throws IOException,
+      URISyntaxException {
+    MiniQJMHACluster qjCluster = null;
+    FileSystem fs = null;
+    try {
+      Builder builder = new MiniQJMHACluster.Builder(conf);
+      builder.getDfsBuilder()
+          .numDataNodes(0);
+      qjCluster = builder.build();
+
+      MiniDFSCluster cluster = qjCluster.getDfsCluster();
+      
+      // No upgrade is in progress at the moment.
+      checkClusterPreviousDirExistence(cluster, false);
+      assertCTimesEqual(cluster);
+      checkJnPreviousDirExistence(qjCluster, false);
+      
+      // Transition NN0 to active and do some FS ops.
+      cluster.transitionToActive(0);
+      fs = HATestUtil.configureFailoverFs(cluster, conf);
+      assertTrue(fs.mkdirs(new Path("/foo1")));
+      
+      // Do the upgrade. Shut down NN1 and then restart NN0 with the upgrade
+      // flag.
+      cluster.shutdownNameNode(1);
+      cluster.getNameNodeInfos()[0].setStartOpt(StartupOption.UPGRADE);
+      cluster.restartNameNode(0, false);
+      
+      checkNnPreviousDirExistence(cluster, 0, true);
+      checkNnPreviousDirExistence(cluster, 1, false);
+      checkJnPreviousDirExistence(qjCluster, true);
+      
+      // NN0 should come up in the active state when given the -upgrade option,
+      // so no need to transition it to active.
+      assertTrue(fs.mkdirs(new Path("/foo2")));
+      
+      // Now bootstrap the standby with the upgraded info.
+      int rc = BootstrapStandby.run(
+          new String[]{"-force"},
+          cluster.getConfiguration(1));
+      assertEquals(0, rc);
+      
+      cluster.restartNameNode(1);
+      
+      checkNnPreviousDirExistence(cluster, 0, true);
+      checkNnPreviousDirExistence(cluster, 1, false);
+      checkJnPreviousDirExistence(qjCluster, true);
+      assertCTimesEqual(cluster);
+      
+      // Shut down the NNs, but deliberately leave the JNs up and running.
+      Collection<URI> nn1NameDirs = cluster.getNameDirs(0);
+      cluster.shutdown();
+
+      conf.setStrings(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, Joiner.on(",").join(nn1NameDirs));
+      NameNode.doRollback(conf, false);
+
+      // The rollback operation should have rolled back the first NN's local
+      // dirs, and the shared dir, but not the other NN's dirs. Those have to be
+      // done by bootstrapping the standby.
+      checkNnPreviousDirExistence(cluster, 0, false);
+      checkJnPreviousDirExistence(qjCluster, false);
+    } finally {
+      if (fs != null) {
+        fs.close();
+      }
+      if (qjCluster != null) {
+        qjCluster.shutdown();
+      }
+    }
+  }
+  
+  /**
+   * Make sure that starting a second NN with the -upgrade flag fails if the
+   * other NN has already done that.
+   */
+  @Test
+  public void testCannotUpgradeSecondNameNode() throws IOException,
+      URISyntaxException {
+    MiniDFSCluster cluster = null;
+    FileSystem fs = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(0)
+      .build();
+  
+      File sharedDir = new File(cluster.getSharedEditsDir(0, 1));
+      
+      // No upgrade is in progress at the moment.
+      checkClusterPreviousDirExistence(cluster, false);
+      assertCTimesEqual(cluster);
+      checkPreviousDirExistence(sharedDir, false);
+      
+      // Transition NN0 to active and do some FS ops.
+      cluster.transitionToActive(0);
+      fs = HATestUtil.configureFailoverFs(cluster, conf);
+      assertTrue(fs.mkdirs(new Path("/foo1")));
+      
+      // Do the upgrade. Shut down NN1 and then restart NN0 with the upgrade
+      // flag.
+      cluster.shutdownNameNode(1);
+      cluster.getNameNodeInfos()[0].setStartOpt(StartupOption.UPGRADE);
+      cluster.restartNameNode(0, false);
+      
+      checkNnPreviousDirExistence(cluster, 0, true);
+      checkNnPreviousDirExistence(cluster, 1, false);
+      checkPreviousDirExistence(sharedDir, true);
+      
+      // NN0 should come up in the active state when given the -upgrade option,
+      // so no need to transition it to active.
+      assertTrue(fs.mkdirs(new Path("/foo2")));
+      
+      // Restart NN0 without the -upgrade flag, to make sure that works.
+      cluster.getNameNodeInfos()[0].setStartOpt(StartupOption.REGULAR);
+      cluster.restartNameNode(0, false);
+      
+      // Make sure we can still do FS ops after upgrading.
+      cluster.transitionToActive(0);
+      assertTrue(fs.mkdirs(new Path("/foo3")));
+      
+      // Make sure that starting the second NN with the -upgrade flag fails.
+      cluster.getNameNodeInfos()[1].setStartOpt(StartupOption.UPGRADE);
+      try {
+        cluster.restartNameNode(1, false);
+        fail("Should not have been able to start second NN with -upgrade");
+      } catch (IOException ioe) {
+        GenericTestUtils.assertExceptionContains(
+            "It looks like the shared log is already being upgraded", ioe);
+      }
+    } finally {
+      if (fs != null) {
+        fs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java

@@ -96,7 +96,7 @@ public class TestInitializeSharedEdits {
     } catch (IOException ioe) {
       LOG.info("Got expected exception", ioe);
       GenericTestUtils.assertExceptionContains(
-          "Cannot start an HA namenode with name dirs that need recovery", ioe);
+          "storage directory does not exist or is not accessible", ioe);
     }
     try {
       cluster.restartNameNode(1, false);
@@ -104,7 +104,7 @@ public class TestInitializeSharedEdits {
     } catch (IOException ioe) {
       LOG.info("Got expected exception", ioe);
       GenericTestUtils.assertExceptionContains(
-          "Cannot start an HA namenode with name dirs that need recovery", ioe);
+          "storage directory does not exist or is not accessible", ioe);
     }
   }
   

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

@@ -17,22 +17,26 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
-import java.util.Random;
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+
 
 import org.apache.hadoop.fs.FileStatus;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.*;
+
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.util.ToolRunner;
 
 public class TestSnapshotFileLength {
 
@@ -112,4 +116,61 @@ public class TestSnapshotFileLength {
     assertThat(bytesRead, is(BLOCKSIZE));
     fis.close();
   }
+
+  /**
+   * Adding as part of jira HDFS-5343
+   * Test for checking the cat command on snapshot path it
+   *  cannot read a file beyond snapshot file length
+   * @throws Exception
+   */
+  @Test (timeout = 600000)
+  public void testSnapshotFileLengthWithCatCommand() throws Exception {
+
+    FSDataInputStream fis = null;
+    FileStatus fileStatus = null;
+
+    int bytesRead;
+    byte[] buffer = new byte[BLOCKSIZE * 8];
+
+    hdfs.mkdirs(sub);
+    Path file1 = new Path(sub, file1Name);
+    DFSTestUtil.createFile(hdfs, file1, BLOCKSIZE, REPLICATION, SEED);
+
+    hdfs.allowSnapshot(sub);
+    hdfs.createSnapshot(sub, snapshot1);
+
+    DFSTestUtil.appendFile(hdfs, file1, BLOCKSIZE);
+
+    // Make sure we can read the entire file via its non-snapshot path.
+    fileStatus = hdfs.getFileStatus(file1);
+    assertEquals(fileStatus.getLen(), BLOCKSIZE * 2);
+    fis = hdfs.open(file1);
+    bytesRead = fis.read(buffer, 0, buffer.length);
+    assertEquals(bytesRead, BLOCKSIZE * 2);
+    fis.close();
+
+    Path file1snap1 =
+        SnapshotTestHelper.getSnapshotPath(sub, snapshot1, file1Name);
+    fis = hdfs.open(file1snap1);
+    fileStatus = hdfs.getFileStatus(file1snap1);
+    assertEquals(fileStatus.getLen(), BLOCKSIZE);
+    // Make sure we can only read up to the snapshot length.
+    bytesRead = fis.read(buffer, 0, buffer.length);
+    assertEquals(bytesRead, BLOCKSIZE);
+    fis.close();
+
+    PrintStream psBackup = System.out;
+    ByteArrayOutputStream bao = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(bao));
+    System.setErr(new PrintStream(bao));
+    // Make sure we can cat the file upto to snapshot length
+    FsShell shell = new FsShell();
+    try{
+      ToolRunner.run(conf, shell, new String[] { "-cat",
+      "/TestSnapshotFileLength/sub1/.snapshot/snapshot1/file1" });
+      assertEquals(bao.size(), BLOCKSIZE);
+    }finally{
+      System.setOut(psBackup);
+    }
+  }
 }

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

@@ -203,6 +203,9 @@ Release 2.4.0 - UNRELEASED
     MAPREDUCE-5672. Provide optional RollingFileAppender for container log4j
     (syslog) (Gera Shegalov via jlowe)
 
+    MAPREDUCE-5725. Make explicit that TestNetworkedJob relies on the Capacity
+    Scheduler (Sandy Ryza)
+
   OPTIMIZATIONS
 
     MAPREDUCE-5484. YarnChild unnecessarily loads job conf twice (Sandy Ryza)
@@ -280,6 +283,14 @@ Release 2.4.0 - UNRELEASED
     MAPREDUCE-5724. JobHistoryServer does not start if HDFS is not running. 
     (tucu)
 
+    MAPREDUCE-5729. mapred job -list throws NPE (kasha)
+
+    MAPREDUCE-5693. Restore MRv1 behavior for log flush (Gera Shegalov via
+    jlowe)
+
+    MAPREDUCE-5723. MR AM container log can be truncated or empty.
+    (Mohammad Kamrul Islam via kasha)
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

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

@@ -27,6 +27,7 @@ import java.net.URI;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.ScheduledExecutorService;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -61,7 +62,6 @@ import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.apache.log4j.LogManager;
 
 /**
  * The main() for MapReduce task processes.
@@ -123,6 +123,7 @@ class YarnChild {
     LOG.debug("PID: " + System.getenv().get("JVM_PID"));
     Task task = null;
     UserGroupInformation childUGI = null;
+    ScheduledExecutorService logSyncer = null;
 
     try {
       int idleLoopCount = 0;
@@ -161,6 +162,8 @@ class YarnChild {
       // set job classloader if configured before invoking the task
       MRApps.setJobClassLoader(job);
 
+      logSyncer = TaskLog.createLogSyncer();
+
       // Create a final reference to the task for the doAs block
       final Task taskFinal = task;
       childUGI.doAs(new PrivilegedExceptionAction<Object>() {
@@ -214,10 +217,7 @@ class YarnChild {
     } finally {
       RPC.stopProxy(umbilical);
       DefaultMetricsSystem.shutdown();
-      // Shutting down log4j of the child-vm...
-      // This assumes that on return from Task.run()
-      // there is no more logging done.
-      LogManager.shutdown();
+      TaskLog.syncLogsShutdown(logSyncer);
     }
   }
 

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

@@ -31,6 +31,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.io.IOUtils;
@@ -45,6 +46,7 @@ import org.apache.hadoop.mapred.FileOutputCommitter;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.LocalContainerLauncher;
 import org.apache.hadoop.mapred.TaskAttemptListenerImpl;
+import org.apache.hadoop.mapred.TaskLog;
 import org.apache.hadoop.mapred.TaskUmbilicalProtocol;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.OutputCommitter;
@@ -119,6 +121,7 @@ import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.service.ServiceOperations;
+import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringInterner;
@@ -212,6 +215,7 @@ public class MRAppMaster extends CompositeService {
   boolean errorHappenedShutDown = false;
   private String shutDownMessage = null;
   JobStateInternal forcedState = null;
+  private final ScheduledExecutorService logSyncer;
 
   private long recoveredJobStartTime = 0;
 
@@ -240,6 +244,7 @@ public class MRAppMaster extends CompositeService {
     this.nmHttpPort = nmHttpPort;
     this.metrics = MRAppMetrics.create();
     this.maxAppAttempts = maxAppAttempts;
+    logSyncer = TaskLog.createLogSyncer();
     LOG.info("Created MRAppMaster for application " + applicationAttemptId);
   }
 
@@ -1078,6 +1083,12 @@ public class MRAppMaster extends CompositeService {
     // All components have started, start the job.
     startJobs();
   }
+  
+  @Override
+  public void stop() {
+    super.stop();
+    TaskLog.syncLogsShutdown(logSyncer);
+  }
 
   private void processRecovery() {
     if (appAttemptID.getAttemptId() == 1) {
@@ -1395,9 +1406,7 @@ public class MRAppMaster extends CompositeService {
       initAndStartAppMaster(appMaster, conf, jobUserName);
     } catch (Throwable t) {
       LOG.fatal("Error starting MRAppMaster", t);
-      System.exit(1);
-    } finally {
-      LogManager.shutdown();
+      ExitUtil.terminate(1, t);
     }
   }
 
@@ -1473,4 +1482,11 @@ public class MRAppMaster extends CompositeService {
       }
     });
   }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    super.serviceStop();
+    LogManager.shutdown();
+  }
+
 }

+ 13 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -445,11 +446,18 @@ public class TypeConverter {
     jobStatus.setStartTime(application.getStartTime());
     jobStatus.setFinishTime(application.getFinishTime());
     jobStatus.setFailureInfo(application.getDiagnostics());
-    jobStatus.setNeededMem(application.getApplicationResourceUsageReport().getNeededResources().getMemory());
-    jobStatus.setNumReservedSlots(application.getApplicationResourceUsageReport().getNumReservedContainers());
-    jobStatus.setNumUsedSlots(application.getApplicationResourceUsageReport().getNumUsedContainers());
-    jobStatus.setReservedMem(application.getApplicationResourceUsageReport().getReservedResources().getMemory());
-    jobStatus.setUsedMem(application.getApplicationResourceUsageReport().getUsedResources().getMemory());
+    ApplicationResourceUsageReport resourceUsageReport =
+        application.getApplicationResourceUsageReport();
+    if (resourceUsageReport != null) {
+      jobStatus.setNeededMem(
+          resourceUsageReport.getNeededResources().getMemory());
+      jobStatus.setNumReservedSlots(
+          resourceUsageReport.getNumReservedContainers());
+      jobStatus.setNumUsedSlots(resourceUsageReport.getNumUsedContainers());
+      jobStatus.setReservedMem(
+          resourceUsageReport.getReservedResources().getMemory());
+      jobStatus.setUsedMem(resourceUsageReport.getUsedResources().getMemory());
+    }
     return jobStatus;
   }
 

+ 9 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java

@@ -23,8 +23,6 @@ import static org.mockito.Mockito.when;
 import java.util.ArrayList;
 import java.util.List;
 
-import junit.framework.Assert;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.JobStatus.State;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -40,6 +38,7 @@ import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.util.Records;
+import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.Mockito;
 
@@ -112,6 +111,14 @@ public class TestTypeConverter {
     when(mockReport.getUser()).thenReturn("dummy-user");
     when(mockReport.getQueue()).thenReturn("dummy-queue");
     String jobFile = "dummy-path/job.xml";
+
+    try {
+      JobStatus status = TypeConverter.fromYarn(mockReport, jobFile);
+    } catch (NullPointerException npe) {
+      Assert.fail("Type converstion from YARN fails for jobs without " +
+          "ApplicationUsageReport");
+    }
+
     ApplicationResourceUsageReport appUsageRpt = Records
         .newRecord(ApplicationResourceUsageReport.class);
     Resource r = Records.newRecord(Resource.class);

+ 87 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLog.java

@@ -23,12 +23,17 @@ import java.io.BufferedReader;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
+import java.io.Flushable;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -44,6 +49,8 @@ import org.apache.hadoop.io.SecureIOUtils;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.util.ProcessTree;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.log4j.Appender;
 import org.apache.log4j.LogManager;
@@ -262,7 +269,86 @@ public class TaskLog {
     }
     writeToIndexFile(logLocation, isCleanup);
   }
-  
+
+  public static synchronized void syncLogsShutdown(
+    ScheduledExecutorService scheduler) 
+  {
+    // flush standard streams
+    //
+    System.out.flush();
+    System.err.flush();
+
+    if (scheduler != null) {
+      scheduler.shutdownNow();
+    }
+
+    // flush & close all appenders
+    LogManager.shutdown(); 
+  }
+
+  @SuppressWarnings("unchecked")
+  public static synchronized void syncLogs() {
+    // flush standard streams
+    //
+    System.out.flush();
+    System.err.flush();
+
+    // flush flushable appenders
+    //
+    final Logger rootLogger = Logger.getRootLogger();
+    flushAppenders(rootLogger);
+    final Enumeration<Logger> allLoggers = rootLogger.getLoggerRepository().
+      getCurrentLoggers();
+    while (allLoggers.hasMoreElements()) {
+      final Logger l = allLoggers.nextElement();
+      flushAppenders(l);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private static void flushAppenders(Logger l) {
+    final Enumeration<Appender> allAppenders = l.getAllAppenders();
+    while (allAppenders.hasMoreElements()) {
+      final Appender a = allAppenders.nextElement();
+      if (a instanceof Flushable) {
+        try {
+          ((Flushable) a).flush();
+        } catch (IOException ioe) {
+          System.err.println(a + ": Failed to flush!"
+            + StringUtils.stringifyException(ioe));
+        }
+      }
+    }
+  }
+
+  public static ScheduledExecutorService createLogSyncer() {
+    final ScheduledExecutorService scheduler =
+      Executors.newSingleThreadScheduledExecutor(
+        new ThreadFactory() {
+          @Override
+          public Thread newThread(Runnable r) {
+            final Thread t = Executors.defaultThreadFactory().newThread(r);
+            t.setDaemon(true);
+            t.setName("Thread for syncLogs");
+            return t;
+          }
+        });
+    ShutdownHookManager.get().addShutdownHook(new Runnable() {
+        @Override
+        public void run() {
+          TaskLog.syncLogsShutdown(scheduler);
+        }
+      }, 50);
+    scheduler.scheduleWithFixedDelay(
+        new Runnable() {
+          @Override
+          public void run() {
+            TaskLog.syncLogs();
+          }
+        }, 0L, 5L, TimeUnit.SECONDS);
+    return scheduler;
+  }
+
   /**
    * The filter for userlogs.
    */

+ 3 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskLogAppender.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.mapred;
 
+import java.io.Flushable;
 import java.util.LinkedList;
 import java.util.Queue;
 
@@ -31,7 +32,7 @@ import org.apache.log4j.spi.LoggingEvent;
  * 
  */
 @InterfaceStability.Unstable
-public class TaskLogAppender extends FileAppender {
+public class TaskLogAppender extends FileAppender implements Flushable {
   private String taskId; //taskId should be managed as String rather than TaskID object
   //so that log4j can configure it from the configuration(log4j.properties). 
   private Integer maxEvents;
@@ -92,6 +93,7 @@ public class TaskLogAppender extends FileAppender {
     }
   }
   
+  @Override
   public void flush() {
     if (qw != null) {
       qw.flush();

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

@@ -44,9 +44,13 @@ import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 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.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.NodeState;
@@ -371,4 +375,29 @@ public class ResourceMgrDelegate extends YarnClient {
       IOException {
     return client.getQueueAclsInfo();
   }
+
+  @Override
+  public ApplicationAttemptReport getApplicationAttemptReport(
+      ApplicationAttemptId appAttemptId) throws YarnException, IOException {
+    return client.getApplicationAttemptReport(appAttemptId);
+  }
+
+  @Override
+  public List<ApplicationAttemptReport> getApplicationAttempts(
+      ApplicationId appId) throws YarnException, IOException {
+    return client.getApplicationAttempts(appId);
+  }
+
+  @Override
+  public ContainerReport getContainerReport(ContainerId containerId)
+      throws YarnException, IOException {
+    return client.getContainerReport(containerId);
+  }
+
+  @Override
+  public List<ContainerReport> getContainers(
+      ApplicationAttemptId applicationAttemptId) throws YarnException,
+      IOException {
+    return client.getContainers(applicationAttemptId);
+  }
 }

+ 14 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestNetworkedJob.java

@@ -45,7 +45,9 @@ import org.apache.hadoop.mapred.lib.IdentityMapper;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus;
 import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.junit.Test;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
@@ -76,8 +78,7 @@ public class TestNetworkedJob {
     FileSystem fileSys = null;
 
     try {
-      mr = MiniMRClientClusterFactory.create(this.getClass(), 2,
-          new Configuration());
+      mr = createMiniClusterWithCapacityScheduler();
 
       JobConf job = new JobConf(mr.getConfig());
 
@@ -129,8 +130,7 @@ public class TestNetworkedJob {
     FileSystem fileSys = null;
 
     try {
-      Configuration conf = new Configuration();
-      mr = MiniMRClientClusterFactory.create(this.getClass(), 2, conf);
+      mr = createMiniClusterWithCapacityScheduler();
 
       JobConf job = new JobConf(mr.getConfig());
 
@@ -315,8 +315,7 @@ public class TestNetworkedJob {
     FileSystem fileSys = null;
     PrintStream oldOut = System.out;
     try {
-      Configuration conf = new Configuration();
-      mr = MiniMRClientClusterFactory.create(this.getClass(), 2, conf);
+      mr = createMiniClusterWithCapacityScheduler();
 
       JobConf job = new JobConf(mr.getConfig());
 
@@ -392,4 +391,13 @@ public class TestNetworkedJob {
       }
     }
   }
+  
+  private MiniMRClientCluster createMiniClusterWithCapacityScheduler()
+      throws IOException {
+    Configuration conf = new Configuration();
+    // Expected queue names depending on Capacity Scheduler queue naming
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        CapacityScheduler.class);
+    return MiniMRClientClusterFactory.create(this.getClass(), 2, conf);
+  }
 }

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

@@ -29,7 +29,6 @@
   <url>http://hadoop.apache.org/mapreduce/</url>
 
   <properties>
-    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <test.logs>true</test.logs>
     <test.timeout>600000</test.timeout>
     <fork.mode>once</fork.mode>

+ 0 - 4
hadoop-project/pom.xml

@@ -39,10 +39,6 @@
     <!-- number of threads/forks to use when running tests in parallel, see parallel-tests profile -->
     <testsThreadCount>4</testsThreadCount>
 
-    <!-- platform encoding override -->
-    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-    <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
-
     <!-- These 2 versions are defined here becuase they are used -->
     <!-- JDIFF generation from embedded ant in the antrun plugin -->
     <jdiff.version>1.0.9</jdiff.version>

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

@@ -33,7 +33,6 @@
   <properties>
     <file.encoding>UTF-8</file.encoding>
     <downloadSources>true</downloadSources>
-    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
   </properties>
 
   <dependencies>

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